Conflict detection for update_deleted in logical replication
Hi hackers,
I am starting a new thread to discuss and propose the conflict detection for
update_deleted scenarios during logical replication. This conflict occurs when
the apply worker cannot find the target tuple to be updated, as the tuple might
have been removed by another origin.
---
BACKGROUND
---
Currently, when the apply worker cannot find the target tuple during an update,
an update_missing conflict is logged. However, to facilitate future automatic
conflict resolution, it has been agreed[1]/messages/by-id/CAJpy0uCov4JfZJeOvY0O21_gk9bcgNUDp4jf8+BbMp+EAv8cVQ@mail.gmail.com[2]/messages/by-id/CAA4eK1Lj-PWrP789KnKxZydisHajd38rSihWXO8MVBLDwxG1Kg@mail.gmail.com that we need to detect both
update_missing and update_deleted conflicts. Specifically, we will detect an
update_deleted conflict if any dead tuple matching the old key value of the
update operation is found; otherwise, it will be classified as update_missing.
Detecting both update_deleted and update_missing conflicts is important for
achieving eventual consistency in a bidirectional cluster, because the
resolution for each conflict type can differs. For example, for an
update_missing conflict, a feasible solution might be converting the update to
an insert and applying it. While for an update_deleted conflict, the preferred
approach could be to skip the update or compare the timestamps of the delete
transactions with the remote update transaction's and choose the most recent
one. For additional context, please refer to [3]/messages/by-id/CAJpy0uC6Zs5WwwiyuvG_kEB6Q3wyDWpya7PXm3SMT_YG=XJJ1w@mail.gmail.com, which gives examples about
how these differences could lead to data divergence.
---
ISSUES and SOLUTION
---
To detect update_deleted conflicts, we need to search for dead tuples in the
table. However, dead tuples can be removed by VACUUM at any time. Therefore, to
ensure consistent and accurate conflict detection, tuples deleted by other
origins must not be removed by VACUUM before the conflict detection process. If
the tuples are removed prematurely, it might lead to incorrect conflict
identification and resolution, causing data divergence between nodes.
Here is an example of how VACUUM could affect conflict detection and how to
prevent this issue. Assume we have a bidirectional cluster with two nodes, A
and B.
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1);
T2: DELETE FROM t WHERE id = 1;
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1;
To retain the deleted tuples, the initial idea was that once transaction T2 had
been applied to both nodes, there was no longer a need to preserve the dead
tuple on Node A. However, a scenario arises where transactions T3 and T2 occur
concurrently, with T3 committing slightly earlier than T2. In this case, if
Node B applies T2 and Node A removes the dead tuple (1,1) via VACUUM, and then
Node A applies T3 after the VACUUM operation, it can only result in an
update_missing conflict. Given that the default resolution for update_missing
conflicts is apply_or_skip (e.g. convert update to insert if possible and apply
the insert), Node A will eventually hold a row (1,2) while Node B becomes
empty, causing data inconsistency.
Therefore, the strategy needs to be expanded as follows: Node A cannot remove
the dead tuple until:
(a) The DELETE operation is replayed on all remote nodes, *AND*
(b) The transactions on logical standbys occurring before the replay of Node
A's DELETE are replayed on Node A as well.
---
THE DESIGN
---
To achieve the above, we plan to allow the logical walsender to maintain and
advance the slot.xmin to protect the data in the user table and introduce a new
logical standby feedback message. This message reports the WAL position that
has been replayed on the logical standby *AND* the changes occurring on the
logical standby before the WAL position are also replayed to the walsender's
node (where the walsender is running). After receiving the new feedback
message, the walsender will advance the slot.xmin based on the flush info,
similar to the advancement of catalog_xmin. Currently, the effective_xmin/xmin
of logical slot are unused during logical replication, so I think it's safe and
won't cause side-effect to reuse the xmin for this feature.
We have introduced a new subscription option (feedback_slots='slot1,...'),
where these slots will be used to check condition (b): the transactions on
logical standbys occurring before the replay of Node A's DELETE are replayed on
Node A as well. Therefore, on Node B, users should specify the slots
corresponding to Node A in this option. The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a feedback
message to the walsender. -- I also thought of making it an automaic way, e.g.
let apply worker select the slots that acquired by the walsenders which connect
to the same remote server(e.g. if apply worker's connection info or some other
flags is same as the walsender's connection info). But it seems tricky because
if some slots are inactive which means the walsenders are not there, the apply
worker could not find the correct slots to check unless we save the host along
with the slot's persistence data.
The new feedback message is sent only if feedback_slots is not NULL. If the
slots in feedback_slots are removed, a final message containing
InvalidXLogRecPtr will be sent to inform the walsender to forget about the
slot.xmin.
To detect update_deleted conflicts during update operations, if the target row
cannot be found, 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.
Please refer to the attached POC patch set which implements above design. The
patch set is split into some parts to make it easier for the initial review.
Please note that each patch is interdependent and cannot work independently.
Thanks a lot to Kuroda-San and Amit for the off-list discussion.
Suggestions and comments are highly appreciated !
[1]: /messages/by-id/CAJpy0uCov4JfZJeOvY0O21_gk9bcgNUDp4jf8+BbMp+EAv8cVQ@mail.gmail.com
[2]: /messages/by-id/CAA4eK1Lj-PWrP789KnKxZydisHajd38rSihWXO8MVBLDwxG1Kg@mail.gmail.com
[3]: /messages/by-id/CAJpy0uC6Zs5WwwiyuvG_kEB6Q3wyDWpya7PXm3SMT_YG=XJJ1w@mail.gmail.com
Best Regards,
Hou Zhijie
Attachments:
v21-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v21-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From bc8d153732678c2abf5ce2e1c7b59ebcbdf6dba6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v21 6/6] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 208 ++++++++++++++++++
2 files changed, 209 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..a35448b861
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,208 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# Create table on node_A
+$node_A->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# Create the same table on node_B
+$node_B->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# 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)");
+
+# 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)");
+
+# 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 initial xmin value of each replication slot is NULL
+is( $node_A->safe_psql(
+ 'postgres',
+ "SELECT xmin IS NULL from pg_replication_slots WHERE slot_name = '$subname_BA'"
+ ),
+ "t",
+ "The initial xmin value of slot '$subname_BA' is NULL");
+
+is( $node_B->safe_psql(
+ 'postgres',
+ "SELECT xmin IS NULL from pg_replication_slots WHERE slot_name = '$subname_AB'"
+ ),
+ "t",
+ "The initial xmin value of slot '$subname_AB' is NULL");
+
+###############################################################################
+# Check that the xmin value of each replication slots will become valid after
+# setting feedback_slots
+###############################################################################
+
+# Log a xl_running_xacts to accelerate the advancement of xmin
+$node_A->safe_psql('postgres', 'SELECT pg_log_standby_snapshot();');
+$node_B->safe_psql('postgres', 'SELECT pg_log_standby_snapshot();');
+
+$node_B->wait_for_catchup($subname_AB);
+$node_A->wait_for_catchup($subname_BA);
+
+$node_B->safe_psql(
+ 'postgres', "
+ ALTER SUBSCRIPTION $subname_BA SET (feedback_slots = '$subname_AB')");
+
+# Insert a record to ensure that the apply worker on Node B reports its flush
+# position. Once the apply worker sends this feedback, the walsender on Node A
+# will update the xmin.
+$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (3);");
+$node_A->wait_for_catchup($subname_BA);
+
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = '$subname_BA'"
+ ),
+ "the xmin value of slot '$subname_BA' becomes valid");
+
+$node_A->safe_psql(
+ 'postgres', "
+ ALTER SUBSCRIPTION $subname_AB SET (feedback_slots = '$subname_BA')");
+
+# Insert a record to ensure that the apply worker on Node A reports its flush
+# position. Once the apply worker sends this feedback, the walsender on Node B
+# will update the xmin.
+$node_B->safe_psql('postgres', "INSERT INTO tab VALUES (4);");
+$node_B->wait_for_catchup($subname_AB);
+
+my $result = $node_A->safe_psql('postgres', "SELECT * FROM tab;");
+is( $result, qq(3
+4),
+ 'check replicated insert on node A'
+);
+
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+is( $result, qq(3
+4),
+ 'check replicated insert on node B'
+);
+
+ok( $node_B->poll_query_until(
+ 'postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = '$subname_AB'"
+ ),
+ "the xmin value of slot '$subname_AB' becomes valid");
+
+# Cleanup the test data
+$node_B->safe_psql('postgres', "DELETE FROM tab;");
+$node_B->wait_for_catchup($subname_AB);
+
+###############################################################################
+# Check that dead tuples on node A cannot be cleaned by VACUUM until the DELETE
+# operations that occurred on the node A have been replayed on the node B, and
+# any changes on node B that occurred before the replay of these DELETE
+# operations are also replayed on node A.
+###############################################################################
+
+# insert a record
+$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1);");
+$node_A->wait_for_catchup($subname_BA);
+
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+is($result, qq(1), '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 a=a+1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab;");
+
+$node_A->wait_for_catchup($subname_BA);
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_missing.*\n.*DETAIL:.* Could not find the row to be deleted.*\n.*Replica identity \(a\)=\(1\)/,
+ 'delete target row is missing 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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the xmin value of each replication slots will become invalid after
+# disabling feedback_slots
+###############################################################################
+
+$node_B->safe_psql(
+ 'postgres', "
+ ALTER SUBSCRIPTION $subname_BA SET (feedback_slots = NONE)");
+
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin IS NULL from pg_replication_slots WHERE slot_name = '$subname_BA'"
+ ),
+ "the xmin value of slot '$subname_BA' becomes invalid");
+
+$node_A->safe_psql(
+ 'postgres', "
+ ALTER SUBSCRIPTION $subname_AB SET (feedback_slots = NONE)");
+
+ok( $node_B->poll_query_until(
+ 'postgres',
+ "SELECT xmin IS NULL from pg_replication_slots WHERE slot_name = '$subname_AB'"
+ ),
+ "the xmin value of slot '$subname_AB' becomes invalid");
+
+done_testing();
--
2.30.0.windows.2
v21-0001-Maintain-and-Advance-slot.xmin-in-logical-walsen.patchapplication/octet-stream; name=v21-0001-Maintain-and-Advance-slot.xmin-in-logical-walsen.patchDownload
From 37537e279e5e84b03ed4a00999f8b757e84adf8e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 12:02:42 +0800
Subject: [PATCH v21 1/6] Maintain and Advance slot.xmin in logical walsender
Conflict detection for update_deleted in logical replication
This set of patches aims to support the detection of an update_deleted
conflict, which occurs 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 removed by VACUUM before conflict detection.
If this happens, a different conflict might be generated and resolved
incorrectly, causing data inconsistency between nodes.
Assuming we have a 2 nodes (A and B) bidirectional cluster and are using last
update win resolution for conflicts:
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1);
T2: DELETE FROM t WHERE id = 1;
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1;
To retain the deleted tuples, the initial idea was that once transaction T2 had
been applied to both nodes, there was no longer a need to preserve the dead
tuple on Node A. However, a scenario arises where transactions T3 and T2 occur
concurrently, with T3 committing slightly earlier than T2. In this case, if
Node B applies T2 and Node A removes the dead tuple (1,1) via VACUUM, and then
Node A applies T3 after the VACUUM operation, it can only result in an
update_missing conflict. Given that the default resolution for update_missing
conflicts is apply_or_skip (e.g. convert update to insert if possible and apply
the insert), Node A will eventually hold a row (1,2) while Node B becomes
empty, causing data inconsistency.
Therefore, the strategy needs to be expanded as follows: Node A cannot remove
the dead tuple until:
(a) The DELETE operation is replayed on all remote nodes, *AND*
(b) The transactions on logical standbys occurring before the replay of Node
A's DELETE are replayed on Node A as well.
To achieve the above, we allow the logical walsender to maintain and advance
the slot.xmin to protect the data in the user table and introduce a new logical
standby feedback message. This message reports the WAL position that has been
replayed on the logical standby *AND* the changes occurring on the logical
standby before the WAL position are also replayed to the walsender's node
(where the walsender is running). After receiving the new feedback message, the
walsender will advance the slot.xmin based on the flush info, similar to the
advancement of catalog_xmin.
We have introduced a new subscription option (feedback_slots='slot1,...'),
where these slots will be used to check condition (b): the transactions on
logical standbys occurring before the replay of Node A's DELETE are replayed on
Node A as well. Therefore, on Node B, users should specify the slots
corresponding to Node A in this option. The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a feedback
message to the walsender.
The new feedback message is sent only if feedback_slots is not NULL. If the
slots in feedback_slots are removed, a final message containing
InvalidXLogRecPtr will be sent to inform the walsender to forget about the
slot.xmin.
To detect update_deleted conflicts during update operations, if the target row
cannot be found, 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.
---
src/backend/replication/logical/logical.c | 26 ++++++++
src/backend/replication/walsender.c | 76 +++++++++++++++++++++++
src/include/replication/slot.h | 34 +++++++---
3 files changed, 129 insertions(+), 7 deletions(-)
diff --git a/src/backend/replication/logical/logical.c b/src/backend/replication/logical/logical.c
index 3fe1774a1e..c42ae2b54e 100644
--- a/src/backend/replication/logical/logical.c
+++ b/src/backend/replication/logical/logical.c
@@ -1695,6 +1695,7 @@ LogicalIncreaseXminForSlot(XLogRecPtr current_lsn, TransactionId xmin)
bool updated_xmin = false;
ReplicationSlot *slot;
bool got_new_xmin = false;
+ bool got_new_data_xmin = false;
slot = MyReplicationSlot;
@@ -1739,6 +1740,27 @@ LogicalIncreaseXminForSlot(XLogRecPtr current_lsn, TransactionId xmin)
*/
got_new_xmin = true;
}
+
+ /*
+ * Don't overwrite if we already have a newer xmin. This can happen if we
+ * restart decoding in a slot.
+ */
+ if (TransactionIdPrecedesOrEquals(xmin, slot->effective_xmin))
+ {
+ }
+
+ /*
+ * Only increase if the previous values have been applied, otherwise we
+ * might never end up updating if the receiver acks too slowly.
+ */
+ else if (slot->candidate_data_xmin_lsn == InvalidXLogRecPtr)
+ {
+ MyReplicationSlot->candidate_xmin = xmin;
+ MyReplicationSlot->candidate_data_xmin_lsn = current_lsn;
+
+ got_new_data_xmin = true;
+ }
+
SpinLockRelease(&slot->mutex);
if (got_new_xmin)
@@ -1748,6 +1770,10 @@ LogicalIncreaseXminForSlot(XLogRecPtr current_lsn, TransactionId xmin)
/* candidate already valid with the current flush position, apply */
if (updated_xmin)
LogicalConfirmReceivedLocation(slot->data.confirmed_flush);
+
+ if (got_new_data_xmin)
+ elog(DEBUG1, "got new data xmin %u at %X/%X", xmin,
+ LSN_FORMAT_ARGS(current_lsn));
}
/*
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index c5f1009f37..9ee88b12f8 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -2319,6 +2319,78 @@ ProcessRepliesIfAny(void)
}
}
+/*
+ * Handle a consumer's confirmation that all changes up to the given LSN have
+ * been received, and that all changes occurring on the consumer side before
+ * the replay of these changes have been confirmed as flushed to the current
+ * node.
+ */
+static void
+ProcessFSFeedbackMessage(void)
+{
+ XLogRecPtr flushPtr;
+ ReplicationSlot *slot = MyReplicationSlot;
+ TransactionId new_xmin;
+
+ Assert(slot != NULL);
+
+ flushPtr = pq_getmsgint64(&reply_message);
+
+ if (XLogRecPtrIsInvalid(flushPtr))
+ {
+ /*
+ * An invalid flush position indicates the end of the feedback,
+ * meaning we don't need to protect user data from being removed.
+ * Therefore, set new_xmin to InvalidTransactionId.
+ */
+ new_xmin = InvalidTransactionId;
+ }
+ else if (!XLogRecPtrIsInvalid(slot->candidate_data_xmin_lsn) &&
+ slot->candidate_data_xmin_lsn <= flushPtr &&
+ TransactionIdIsValid(slot->candidate_xmin) &&
+ slot->data.xmin != slot->candidate_xmin)
+ {
+ new_xmin = slot->candidate_xmin;
+ }
+ else
+ {
+ /* No candidate xmin can be used or the xmin is unchanged, so exit. */
+ return;
+ }
+
+ /*
+ * We have to write the changed xmin to disk *before* we change the
+ * in-memory value, otherwise after a crash we wouldn't know that some
+ * catalog tuples might have been removed already.
+ *
+ * Ensure that by first writing to ->xmin and only update ->effective_xmin
+ * once the new state is synced to disk. After a crash ->effective_xmin is
+ * set to ->xmin.
+ */
+ SpinLockAcquire(&slot->mutex);
+ slot->data.xmin = new_xmin;
+ slot->candidate_data_xmin_lsn = InvalidXLogRecPtr;
+ slot->candidate_xmin = InvalidTransactionId;
+ SpinLockRelease(&slot->mutex);
+
+ /* first write new xmin to disk, so we know what's up after a crash */
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ elog(DEBUG1, "updated xmin: %u", new_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);
+ slot->effective_xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ ReplicationSlotsComputeRequiredXmin(false);
+}
+
/*
* Process a status update message received from standby.
*/
@@ -2342,6 +2414,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'x':
+ ProcessFSFeedbackMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..f1d8875d40 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -166,13 +166,17 @@ typedef struct ReplicationSlot
/*
* For logical decoding, it's extremely important that we never remove any
* data that's still needed for decoding purposes, even after a crash;
- * otherwise, decoding will produce wrong answers. Ordinary streaming
- * replication also needs to prevent old row versions from being removed
- * too soon, but the worst consequence we might encounter there is
- * unwanted query cancellations on the standby. Thus, for logical
- * decoding, this value represents the latest xmin that has actually been
- * written to disk, whereas for streaming replication, it's just the same
- * as the persistent value (data.xmin).
+ * otherwise, decoding will produce wrong answers. Both logical
+ * replication conflict detection and ordinary streaming replication needs
+ * to prevent old row versions from being removed too soon. The worst
+ * consequence in ordinary streaming replication would be unwanted query
+ * cancellations on the standby. However, for logical conflict detection,
+ * it is essential to identify the origin and timestamp of old row
+ * versions to correctly detect and resolve conflicts. Otherwise, it could
+ * cause data inconsistency between nodes. For logical decoding and
+ * replication, this value represents the latest xmin that has actually
+ * been written to disk, whereas for streaming replication, it's just the
+ * same as the persistent value (data.xmin).
*/
TransactionId effective_xmin;
TransactionId effective_catalog_xmin;
@@ -198,6 +202,22 @@ typedef struct ReplicationSlot
XLogRecPtr candidate_restart_valid;
XLogRecPtr candidate_restart_lsn;
+ /*
+ * If the client has sent feedback confirming that the WAL position
+ * flushed to a remote node (corresponding to the feedback slot on the
+ * client) is greater than or equal to the candidate_data_xmin_lsn, we can
+ * advance the xmin.
+ *
+ * This mechanism is used for conflict detection in a bidirectional
+ * logical replication cluster. It ensures that dead tuples cannot be
+ * cleaned by VACUUM until the DELETE operations that occurred on the
+ * local node have been replayed on the subscriber, and any changes on
+ * remote nodes that occurred before the replay of these DELETE operations
+ * are also replayed locally.
+ */
+ TransactionId candidate_xmin;
+ XLogRecPtr candidate_data_xmin_lsn;
+
/*
* This value tracks the last confirmed_flush LSN flushed which is used
* during a shutdown checkpoint to decide if logical's slot data should be
--
2.30.0.windows.2
v21-0002-Add-a-subscription-option-feedback_slots.patchapplication/octet-stream; name=v21-0002-Add-a-subscription-option-feedback_slots.patchDownload
From 094b607c1183fd6cfbef7ccef14fe074c1129799 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 29 Aug 2024 17:28:46 +0800
Subject: [PATCH v21 2/6] Add a subscription option feedback_slots
This patch adds a new subscription option (feedback_slots='slot1,...'), which
accepts valid logical slot. User can specify the option when executing
CREATE SUBSCRIPTION or ALTER SUBSCRIPTION.
TODO: doc.
---
src/backend/catalog/pg_subscription.c | 11 ++
src/backend/catalog/system_views.sql | 2 +-
src/backend/commands/subscriptioncmds.c | 184 ++++++++++++++++++++-
src/bin/pg_dump/pg_dump.c | 50 +++++-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 7 +-
src/bin/psql/tab-complete.c | 14 +-
src/include/catalog/pg_subscription.h | 14 ++
src/include/replication/worker_internal.h | 4 +
src/test/regress/expected/subscription.out | 170 ++++++++++---------
src/test/regress/sql/subscription.sql | 11 ++
11 files changed, 377 insertions(+), 91 deletions(-)
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 9efc9159f2..dfe1e96cab 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -107,6 +107,17 @@ GetSubscription(Oid subid, bool missing_ok)
Anum_pg_subscription_suborigin);
sub->origin = TextDatumGetCString(datum);
+ /* Get feedback slots */
+ datum = SysCacheGetAttr(SUBSCRIPTIONOID,
+ tup,
+ Anum_pg_subscription_subfeedbackslots,
+ &isnull);
+
+ if (!isnull)
+ sub->feedback_slots = textarray_to_stringlist(DatumGetArrayTypeP(datum));
+ else
+ sub->feedback_slots = NIL;
+
/* Is the subscription owner a superuser? */
sub->ownersuperuser = superuser_arg(sub->owner);
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 7fd5d256a1..762f397cc7 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1356,7 +1356,7 @@ 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)
+ subslotname, subsynccommit, subpublications, suborigin, subfeedbackslots)
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 02ccc636b8..48ce766e0e 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -52,6 +52,7 @@
#include "utils/memutils.h"
#include "utils/pg_lsn.h"
#include "utils/syscache.h"
+#include "utils/varlena.h"
/*
* Options that can be specified by the user in CREATE/ALTER SUBSCRIPTION
@@ -73,6 +74,7 @@
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_LSN 0x00004000
#define SUBOPT_ORIGIN 0x00008000
+#define SUBOPT_FEEDBACK_SLOTS 0x00010000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -99,6 +101,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
char *origin;
+ List *feedback_slots;
XLogRecPtr lsn;
} SubOpts;
@@ -164,6 +167,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
+ if (IsSet(supported_opts, SUBOPT_FEEDBACK_SLOTS))
+ opts->feedback_slots = NULL;
/* Parse options */
foreach(lc, stmt_options)
@@ -330,6 +335,72 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("unrecognized origin value: \"%s\"", opts->origin));
}
+ else if (IsSet(supported_opts, SUBOPT_FEEDBACK_SLOTS) &&
+ strcmp(defel->defname, "feedback_slots") == 0)
+ {
+ char *feedback_slots;
+ char *rawname;
+ ListCell *cell;
+
+ if (IsSet(opts->specified_opts, SUBOPT_FEEDBACK_SLOTS))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_FEEDBACK_SLOTS;
+ feedback_slots = defGetString(defel);
+
+ /* Setting feedback_slots = NONE is treated as no feedback slots */
+ if (strcmp(feedback_slots, "none") == 0)
+ {
+ opts->feedback_slots = NIL;
+
+ /* Skip the slot validation */
+ continue;
+ }
+
+ /* Need a modifiable copy of the option string */
+ rawname = pstrdup(feedback_slots);
+
+ /* Parse string into a list of slots */
+ if (!SplitIdentifierString(rawname, ',', &opts->feedback_slots))
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("parameter \"%s\" must be a list of slot names",
+ "feedback_slots")));
+
+ /*
+ * First, validate the slot name and check for duplicate slot
+ * names.
+ */
+ foreach(cell, opts->feedback_slots)
+ {
+ char *name = lfirst(cell);
+ ListCell *scell;
+
+ ReplicationSlotValidateName(name, ERROR);
+
+ foreach(scell, opts->feedback_slots)
+ {
+ char *sname = lfirst(scell);
+
+ if (scell == cell)
+ break;
+
+ if (strcmp(name, sname) == 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_DUPLICATE_OBJECT),
+ errmsg("slot name \"%s\" used more than once",
+ sname)));
+ }
+ }
+
+ LWLockAcquire(ReplicationSlotControlLock, LW_SHARED);
+
+ /* Next, verify if the slot is a valid logical slot */
+ foreach_ptr(char, name, opts->feedback_slots)
+ (void) ValidateAndGetFeedbackSlot(name);
+
+ LWLockRelease(ReplicationSlotControlLock);
+ }
else if (IsSet(supported_opts, SUBOPT_LSN) &&
strcmp(defel->defname, "lsn") == 0)
{
@@ -439,6 +510,77 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
}
}
+/*
+ * Return the replication slot specified in the feedback_slots GUC parameter.
+ *
+ * This function checks the provided slot and issues warnings if the slot does
+ * not exist, or is a physical replication slot or is invalidated.
+ *
+ * Note that this function does not report errors because it is common for the
+ * subscription on the remote node to be created after this validation is
+ * performed. Consequently, the corresponding logical slot may not yet exist
+ * locally.
+ */
+ReplicationSlot *
+ValidateAndGetFeedbackSlot(char *slotname)
+{
+ bool invalidated;
+ ReplicationSlot *slot;
+
+ slot = SearchNamedReplicationSlot(slotname, false);
+
+ if (!slot)
+ {
+ /*
+ * If a slot name provided in feedback_slots does not exist, report a
+ * message and exit the loop.
+ */
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("replication slot \"%s\" specified in subscription parameter %s does not exist",
+ slotname, "feedback_slots"),
+ errhint("Consider creating the slot \"%s\" or amend subscription parameter %s.",
+ slotname, "feedback_slots"));
+
+ return NULL;
+ }
+
+ if (SlotIsPhysical(slot))
+ {
+ /*
+ * If a logical slot name is provided in feedback_slots, report a
+ * message and exit the loop.
+ */
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("cannot have physical replication slot \"%s\" in subscription parameter %s",
+ slotname, "feedback_slots"),
+ errhint("Consider removing logical slot \"%s\" from subscription parameter %s.",
+ slotname, "feedback_slots"));
+
+ return NULL;
+ }
+
+ SpinLockAcquire(&slot->mutex);
+ invalidated = slot->data.invalidated != RS_INVAL_NONE;
+ SpinLockRelease(&slot->mutex);
+
+ if (invalidated)
+ {
+ /* Specified logical slot has been invalidated */
+ ereport(WARNING,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("slot \"%s\" specified in subscription parameter %s has been invalidated",
+ slotname, "feedback_slots"),
+ errhint("Consider dropping and recreating the slot \"%s\" or amend subscription parameter %s.",
+ slotname, "feedback_slots"));
+
+ return NULL;
+ }
+
+ return slot;
+}
+
/*
* Add publication names from the list to a string.
*/
@@ -563,6 +705,26 @@ publicationListToArray(List *publist)
return PointerGetDatum(arr);
}
+/*
+ * Similar to publicationListToArray(), but used for feedback_slots.
+ */
+static Datum
+FeedbackSlotListToArray(List *feedbacklist)
+{
+ ArrayType *arr;
+ Datum *datums;
+ int i = 0;
+
+ datums = (Datum *) palloc(sizeof(Datum) * list_length(feedbacklist));
+
+ foreach_ptr(char, name, feedbacklist)
+ datums[i++] = CStringGetTextDatum(name);
+
+ arr = construct_array_builtin(datums, list_length(feedbacklist), TEXTOID);
+
+ return PointerGetDatum(arr);
+}
+
/*
* Create new subscription.
*/
@@ -594,7 +756,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_ORIGIN |
+ SUBOPT_FEEDBACK_SLOTS);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -715,6 +878,12 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_suborigin - 1] =
CStringGetTextDatum(opts.origin);
+ if (opts.feedback_slots != NIL)
+ values[Anum_pg_subscription_subfeedbackslots - 1] =
+ FeedbackSlotListToArray(opts.feedback_slots);
+ else
+ nulls[Anum_pg_subscription_subfeedbackslots - 1] = true;
+
tup = heap_form_tuple(RelationGetDescr(rel), values, nulls);
/* Insert tuple into catalog. */
@@ -1196,7 +1365,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_ORIGIN | SUBOPT_FEEDBACK_SLOTS);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1363,6 +1532,17 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_suborigin - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_FEEDBACK_SLOTS))
+ {
+ if (opts.feedback_slots != NIL)
+ values[Anum_pg_subscription_subfeedbackslots - 1] =
+ FeedbackSlotListToArray(opts.feedback_slots);
+ else
+ nulls[Anum_pg_subscription_subfeedbackslots - 1] = true;
+
+ replaces[Anum_pg_subscription_subfeedbackslots - 1] = true;
+ }
+
update_tuple = true;
break;
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index dacb033e98..1b71749f2b 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4847,6 +4847,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subfeedbackslots;
int i,
ntups;
@@ -4919,10 +4920,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subfeedbackslots\n");
+ else
+ appendPQExpBuffer(query,
+ " NULL AS subfeedbackslots\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4962,6 +4970,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subfeedbackslots = PQfnumber(res, "subfeedbackslots");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5008,6 +5017,11 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ if (PQgetisnull(res, i, i_subfeedbackslots))
+ subinfo[i].subfeedbackslots = NULL;
+ else
+ subinfo[i].subfeedbackslots =
+ pg_strdup(PQgetvalue(res, i, i_subfeedbackslots));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5184,9 +5198,11 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
PQExpBuffer delq;
PQExpBuffer query;
PQExpBuffer publications;
+ PQExpBuffer feedbackslots;
char *qsubname;
char **pubnames = NULL;
- int npubnames = 0;
+ char **feedbackslotnames = NULL;
+ int nentries = 0;
int i;
char two_phase_disabled[] = {LOGICALREP_TWOPHASE_STATE_DISABLED, '\0'};
@@ -5207,11 +5223,11 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
appendStringLiteralAH(query, subinfo->subconninfo, fout);
/* Build list of quoted publications and append them to query. */
- if (!parsePGArray(subinfo->subpublications, &pubnames, &npubnames))
+ if (!parsePGArray(subinfo->subpublications, &pubnames, &nentries))
pg_fatal("could not parse %s array", "subpublications");
publications = createPQExpBuffer();
- for (i = 0; i < npubnames; i++)
+ for (i = 0; i < nentries; i++)
{
if (i > 0)
appendPQExpBufferStr(publications, ", ");
@@ -5219,6 +5235,25 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
appendPQExpBufferStr(publications, fmtId(pubnames[i]));
}
+ nentries = 0;
+
+ /*
+ * Do same thing for feedback slots. One difference is that
+ * subfeedbackslots is nullable.
+ */
+ if (subinfo->subfeedbackslots != NULL &&
+ !parsePGArray(subinfo->subfeedbackslots, &feedbackslotnames, &nentries))
+ pg_fatal("could not parse %s array", "subfeedbackslots");
+
+ feedbackslots = createPQExpBuffer();
+ for (i = 0; i < nentries; i++)
+ {
+ if (i > 0)
+ appendPQExpBufferStr(feedbackslots, ", ");
+
+ appendPQExpBufferStr(feedbackslots, fmtId(feedbackslotnames[i]));
+ }
+
appendPQExpBuffer(query, " PUBLICATION %s WITH (connect = false, slot_name = ", publications->data);
if (subinfo->subslotname)
appendStringLiteralAH(query, subinfo->subslotname, fout);
@@ -5254,6 +5289,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (pg_strcasecmp(subinfo->suborigin, LOGICALREP_ORIGIN_ANY) != 0)
appendPQExpBuffer(query, ", origin = %s", subinfo->suborigin);
+ if (feedbackslots->len > 0)
+ appendPQExpBuffer(query, ", subfeedbackslots = \'%s\'", feedbackslots->data);
+
appendPQExpBufferStr(query, ");\n");
/*
@@ -5314,6 +5352,8 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
destroyPQExpBuffer(publications);
free(pubnames);
+ destroyPQExpBuffer(feedbackslots);
+ free(feedbackslotnames);
destroyPQExpBuffer(delq);
destroyPQExpBuffer(query);
diff --git a/src/bin/pg_dump/pg_dump.h b/src/bin/pg_dump/pg_dump.h
index 0b7d21b2e9..e2ae3d3b08 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -672,6 +672,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subfeedbackslots;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7c9a1f234c..d8cc331f0d 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6539,7 +6539,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)
{
@@ -6608,6 +6608,11 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subfeedbackslots AS \"%s\"\n",
+ gettext_noop("Feedback slots"));
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a7ccde6d7d..d3d9757ecd 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1946,9 +1946,10 @@ psql_completion(const char *text, int start, int end)
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (HeadMatches("ALTER", "SUBSCRIPTION", MatchAny) && TailMatches("SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "run_as_owner", "slot_name",
- "streaming", "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "feedback_slots", "origin", "password_required",
+ "run_as_owner", "slot_name", "streaming",
+ "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (HeadMatches("ALTER", "SUBSCRIPTION", MatchAny) && TailMatches("SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3343,9 +3344,10 @@ psql_completion(const char *text, int start, int end)
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (HeadMatches("CREATE", "SUBSCRIPTION") && TailMatches("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");
+ "disable_on_error", "enabled", "failover",
+ "feedback_slots", "origin", "password_required",
+ "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 0aa14ec4a2..b96bca15fc 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -113,6 +113,14 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
/* Only publish data originating from the specified origin */
text suborigin BKI_DEFAULT(LOGICALREP_ORIGIN_ANY);
+
+ /*
+ * Send feedback to the publisher about the WAL position that has been
+ * replayed, and that all changes that occurred before the replay of the
+ * WAL have been flushed to the remote node corresponding to the specified
+ * logical slots.
+ */
+ text subfeedbackslots[1] BKI_FORCE_NULL;
#endif
} FormData_pg_subscription;
@@ -157,6 +165,12 @@ typedef struct Subscription
List *publications; /* List of publication names to subscribe to */
char *origin; /* Only publish data originating from the
* specified origin */
+ List *feedback_slots; /* Send feedback to the publisher about the
+ * WAL position that has been replayed, and
+ * that all changes that occurred before the
+ * replay of the WAL have been flushed to the
+ * remote node corresponding to the specified
+ * logical slots. */
} Subscription;
/* Disallow streaming in-progress transactions. */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..836c0a4798 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -326,6 +326,10 @@ extern void pa_decr_and_wait_stream_block(void);
extern void pa_xact_finish(ParallelApplyWorkerInfo *winfo,
XLogRecPtr remote_lsn);
+struct ReplicationSlot;
+
+extern struct ReplicationSlot *ValidateAndGetFeedbackSlot(char *slotname);
+
#define isParallelApplyWorker(worker) ((worker)->in_use && \
(worker)->type == WORKERTYPE_PARALLEL_APPLY)
#define isTablesyncWorker(worker) ((worker)->in_use && \
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 17d48b1685..640c25f661 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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | d | f | none | t | 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 | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | 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 | 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 | Feedback slots | 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
(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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | d | f | any | f | t | 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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | d | f | any | t | 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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | d | f | any | t | 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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | off | d | f | any | t | 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 | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | off | d | f | any | t | 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 | 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 | Feedback slots | 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
(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 | Feedback slots | 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
(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 | Feedback slots | 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
(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 | Feedback slots | 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
(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 | Feedback slots | 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
(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 | Feedback slots | 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
(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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | p | f | any | t | 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 | Feedback slots | 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
(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 | Feedback slots | 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
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,36 @@ 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 | 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 | Feedback slots | 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
(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 | off | 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 | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | t | any | t | f | f | | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- fail - cannot set duplicated slot name as feedback_slots
+ALTER SUBSCRIPTION regress_testsub SET (feedback_slots = 'slot1, slot1');
+ERROR: slot name "slot1" used more than once
+-- ok - feedback_slots accepts NONE
+ALTER SUBSCRIPTION regress_testsub SET (feedback_slots = NONE);
+-- ok - feedback_slots can be set even when given slots do not exist
+ALTER SUBSCRIPTION regress_testsub SET (feedback_slots = 'slot1, slot2');
+WARNING: replication slot "slot1" specified in subscription parameter feedback_slots does not exist
+HINT: Consider creating the slot "slot1" or amend subscription parameter feedback_slots.
+WARNING: replication slot "slot2" specified in subscription parameter feedback_slots does not exist
+HINT: Consider creating the slot "slot2" or amend subscription parameter feedback_slots.
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Feedback slots | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | t | any | t | f | f | {slot1,slot2} | 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 007c9e7037..ed5e1132c5 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -284,6 +284,17 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
\dRs+
+-- fail - cannot set duplicated slot name as feedback_slots
+ALTER SUBSCRIPTION regress_testsub SET (feedback_slots = 'slot1, slot1');
+
+-- ok - feedback_slots accepts NONE
+ALTER SUBSCRIPTION regress_testsub SET (feedback_slots = NONE);
+
+-- ok - feedback_slots can be set even when given slots do not exist
+ALTER SUBSCRIPTION regress_testsub SET (feedback_slots = 'slot1, slot2');
+
+\dRs+
+
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
--
2.30.0.windows.2
v21-0003-Send-the-slot-flush-feedback-message-via-apply-w.patchapplication/octet-stream; name=v21-0003-Send-the-slot-flush-feedback-message-via-apply-w.patchDownload
From 0d10ca479a335fd6d6aee63a77a003b6fee76f14 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 12:03:57 +0800
Subject: [PATCH v21 3/6] Send the slot flush feedback message via apply worker
This patch allows the apply worker to send feedback about the WAL position of
the publisher that has been flushed locally and that all local changes
occurring before this WAL position have been flushed to the remote node
corresponding to the feedback slots. In this context, the remote node indicates
the publisher, as the feedback is currently used only in a bidirectional
cluster to preserve old row versions for conflict detection purposes.
If feedback_slots is NULL and a status might have already been sent to update
the xmin value of the slot, an InvalidXLogRecPtr is sent. This indicates that
the apply worker no longer tracks or sends feedback about the confirmed flush
position. In this case, the publisher should reset the slot's xmin to
InvalidXLogRecPtr to allow dead tuples in user tables to be removed.
The status message is sent at most once per wal_receiver_status_interval.
TODO: doc.
---
src/backend/replication/logical/worker.c | 262 ++++++++++++++++++++++-
1 file changed, 260 insertions(+), 2 deletions(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0fb577d328..f4a31ef534 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"
@@ -378,6 +379,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void send_flush_status_feedback(XLogRecPtr recvpos,
+ bool feedback_slots_changed);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3499,8 +3502,18 @@ get_flush_position(XLogRecPtr *write, XLogRecPtr *flush,
if (pos->local_end <= local_flush)
{
*flush = pos->remote_end;
- dlist_delete(iter.cur);
- pfree(pos);
+
+ /*
+ * If feedback slots are specified, do not clean up entries here.
+ * These entries are needed to compute positions that are
+ * confirmed to be flushed to the remote node. The entries will be
+ * cleaned up in the get_slot_flush_position().
+ */
+ if (!MySubscription->feedback_slots)
+ {
+ dlist_delete(iter.cur);
+ pfree(pos);
+ }
}
else
{
@@ -3520,6 +3533,114 @@ get_flush_position(XLogRecPtr *write, XLogRecPtr *flush,
*have_pending_txes = !dlist_is_empty(&lsn_mapping);
}
+
+/*
+ * Get the oldest confirmed flush LSN across all slots specified in
+ * feedback_slots.
+ */
+static XLogRecPtr
+get_slot_confirmed_flush(void)
+{
+ int checked_slot_num = 0;
+ XLogRecPtr min_confirmed_flush = InvalidXLogRecPtr;
+
+ /* Quickly exit if there are no feedback slots configured */
+ if (!MySubscription->feedback_slots)
+ return InvalidXLogRecPtr;
+
+ /*
+ * To prevent concurrent slot dropping and creation while filtering the
+ * slots, take the ReplicationSlotControlLock outside of the loop.
+ */
+ LWLockAcquire(ReplicationSlotControlLock, LW_SHARED);
+
+ foreach_ptr(String, name, MySubscription->feedback_slots)
+ {
+ XLogRecPtr confirmed_flush;
+ ReplicationSlot *slot;
+
+ slot = ValidateAndGetFeedbackSlot(strVal(name));
+
+ if (!slot)
+ break;
+
+ SpinLockAcquire(&slot->mutex);
+ confirmed_flush = slot->data.confirmed_flush;
+ SpinLockRelease(&slot->mutex);
+
+ if (XLogRecPtrIsInvalid(min_confirmed_flush) ||
+ min_confirmed_flush > confirmed_flush)
+ min_confirmed_flush = confirmed_flush;
+
+ checked_slot_num++;
+ }
+
+ LWLockRelease(ReplicationSlotControlLock);
+
+ if (checked_slot_num != list_length(MySubscription->feedback_slots))
+ return InvalidXLogRecPtr;
+
+ return min_confirmed_flush;
+}
+
+/*
+ * Figure out the WAL positions confirmed to be flushed to the remote node
+ * corresponding to the feedback slots, to report to the walsender process.
+ *
+ * When reporting the position to the sender, this function iterates through
+ * the list and checks which entries are confirmed to be flushed to the remote
+ * node. These entries are then reported as having been flushed.
+ *
+ * The have_pending_txes variable is set to true if there are outstanding
+ * transactions that need to be flushed.
+ */
+static void
+get_slot_flush_position(XLogRecPtr *slot_flush, bool *have_pending_txes)
+{
+ dlist_mutable_iter iter;
+ XLogRecPtr slot_confirmed_flush;
+
+ *slot_flush = InvalidXLogRecPtr;
+
+ slot_confirmed_flush = get_slot_confirmed_flush();
+
+ /*
+ * There is no need to iterate the list if we cannot get the correct slot
+ * flush position.
+ */
+ if (XLogRecPtrIsInvalid(slot_confirmed_flush))
+ {
+ *have_pending_txes = !dlist_is_empty(&lsn_mapping);
+ return;
+ }
+
+ dlist_foreach_modify(iter, &lsn_mapping)
+ {
+ FlushPosition *pos =
+ dlist_container(FlushPosition, node, iter.cur);
+
+ if (pos->local_end <= slot_confirmed_flush)
+ {
+ Assert(pos->local_end <= GetFlushRecPtr(NULL));
+
+ *slot_flush = pos->remote_end;
+ dlist_delete(iter.cur);
+ pfree(pos);
+ }
+ else
+ {
+ /*
+ * Don't want to uselessly iterate over the rest of the list which
+ * could potentially be long.
+ */
+ *have_pending_txes = true;
+ return;
+ }
+ }
+
+ *have_pending_txes = !dlist_is_empty(&lsn_mapping);
+}
+
/*
* Store current remote/local lsn pair in the tracking list.
*/
@@ -3692,6 +3813,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
+
+ send_flush_status_feedback(last_received, false);
}
/* other message types are purposefully ignored */
@@ -3705,6 +3828,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ send_flush_status_feedback(last_received, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3915,6 +4040,122 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
last_flushpos = flushpos;
}
+/*
+ * Send a Flush Status Update message to the server.
+ *
+ * This function sends the WAL position of the publisher that has been flushed
+ * locally and confirms that all local changes occurring before this WAL
+ * position have been flushed to the remote node corresponding to the feedback
+ * slots. In this context, the remote node indicates the publisher, as the
+ * feedback is currently used only in a bidirectional cluster to preserve old
+ * row versions for conflict detection purposes (see the comments for
+ * ReplicationSlot in slot.h for details).
+ *
+ * If feedback_slots is NULL and a status might have already been sent to
+ * update the xmin value of the slot, an InvalidXLogRecPtr is sent. This
+ * indicates that the apply worker no longer tracks or sends feedback about the
+ * confirmed flush position. In this case, the publisher should reset the
+ * slot's xmin to InvalidXLogRecPtr to allow dead tuples in user tables to be
+ * removed.
+ *
+ * The status message is sent at most once per wal_receiver_status_interval,
+ * unless 'feedback_slots_changed' is set to true. In that case, we recalculate
+ * the sending time.
+ */
+static void
+send_flush_status_feedback(XLogRecPtr recvpos, bool feedback_slots_changed)
+{
+ static StringInfo reply_message = NULL;
+ static TimestampTz send_time = 0;
+
+ /*
+ * Initialize the flag to force sending a final message upon the first
+ * connection if feedback_slots is not specified. This is necessary
+ * because a previous connection might have set xmin on a replication slot
+ * on the publisher.
+ */
+ static bool publisher_has_slot_xmin = true;
+
+ XLogRecPtr slot_flushpos;
+ bool have_pending_txes;
+ TimestampTz now;
+
+ /* Reset the last-send time when the feedback_slots option was changed */
+ if (feedback_slots_changed)
+ send_time = 0;
+
+ /*
+ * Return if status reporting is disabled unless a status has already been
+ * sent. In such a case, one more message is needed to inform the
+ * walsender to forget about the xmin.
+ */
+ if ((wal_receiver_status_interval <= 0 || !MySubscription->feedback_slots) &&
+ !publisher_has_slot_xmin)
+ return;
+
+ get_slot_flush_position(&slot_flushpos, &have_pending_txes);
+
+ /*
+ * If there are no outstanding transactions to flush and feedback has been
+ * sent previously, we can report the latest received position (recvpos).
+ *
+ * Note that we cannot report the latest position if no feedback has ever
+ * been sent. This is because all the entries in lsn_mapping could have
+ * been cleaned up before specifying feedback slots. As a result, we
+ * cannot determine the local position corresponding to previous local
+ * flushed transactions, and thus cannot confirm if the WALs have been
+ * flushed to the remote node.
+ *
+ * When a feedback message has been sent, it means we have confirmed that
+ * all changes applied after specifying feedback slots have been flushed
+ * on the remote node.
+ */
+ if (MySubscription->feedback_slots && send_time && !have_pending_txes)
+ slot_flushpos = recvpos;
+
+ /*
+ * Return if feedback slots are specified but the slot's flush position is
+ * invalid. This indicates that some specified slots are not valid
+ * feedback slots (see ValidateAndGetFeedbackSlot for details), or the
+ * worker has has not yet applied any changes since the feedback slots
+ * were specified.
+ */
+ if (MySubscription->feedback_slots && XLogRecPtrIsInvalid(slot_flushpos))
+ return;
+
+ Assert(!XLogRecPtrIsInvalid(slot_flushpos) || publisher_has_slot_xmin);
+
+ /* Get current timestamp */
+ now = GetCurrentTimestamp();
+
+ /* Send feedback at most once per wal_receiver_status_interval */
+ if (!TimestampDifferenceExceeds(send_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'x');
+ pq_sendint64(reply_message, slot_flushpos);
+
+ elog(DEBUG2, "sending flush status feedback %X/%X",
+ LSN_FORMAT_ARGS(slot_flushpos));
+
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ publisher_has_slot_xmin = !XLogRecPtrIsInvalid(slot_flushpos);
+ send_time = XLogRecPtrIsInvalid(slot_flushpos) ? 0 : now;
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -3955,6 +4196,7 @@ maybe_reread_subscription(void)
MemoryContext oldctx;
Subscription *newsub;
bool started_tx = false;
+ bool send_feedback = false;
/* When cache state is valid there is nothing to do here. */
if (MySubscriptionValid)
@@ -4005,6 +4247,19 @@ maybe_reread_subscription(void)
/* two-phase cannot be altered while the worker is running */
Assert(newsub->twophasestate == MySubscription->twophasestate);
+ /*
+ * Force sending a message when the feedback_slots parameter changes. If
+ * feedback_slots is changed to NULL, send a final message to reset the
+ * xmin of the slot on the publisher. Otherwise, try to send a message to
+ * update the slot's xmin on the publisher.
+ *
+ * The message will be sent after switching to the new subscription info
+ * (at the end of the function) so that the updated feedback slots info
+ * can be used when sending the message.
+ */
+ if (!equal(MySubscription->feedback_slots, newsub->feedback_slots))
+ send_feedback = true;
+
/*
* Exit if any parameter that affects the remote connection was changed.
* The launcher will start a new worker but note that the parallel apply
@@ -4072,6 +4327,9 @@ maybe_reread_subscription(void)
if (started_tx)
CommitTransactionCommand();
+ if (send_feedback)
+ send_flush_status_feedback(MyLogicalRepWorker->last_lsn, true);
+
MySubscriptionValid = true;
}
--
2.30.0.windows.2
v21-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v21-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 42e325e2d833fb2dfebade4cbcca4827ab967976 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 2 Sep 2024 17:40:56 +0800
Subject: [PATCH v21 4/6] 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.
---
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 136 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 73 ++++++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
9 files changed, 227 insertions(+), 33 deletions(-)
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 762f397cc7..3fa2b2eb65 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1369,6 +1369,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 1086cbc962..d19266b305 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,9 @@ tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
if (att->attisdropped || att->attgenerated)
continue;
+ if (columns && !bms_is_member(att->attnum, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +429,7 @@ retry:
/* 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;
@@ -474,6 +480,128 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * Search the relation 'rel' for the most recently deleted tuple that matches
+ * the values in 'searchslot' and is not yet removable by VACUUM.
+ *
+ * The commit timestamp of the transaction that deleted the tuple is used to
+ * determine whether the tuple is the most recently deleted one.
+ *
+ * This operation can be quite slow on tables with a large number of rows.
+ * However, it is primarily used in rare conflict cases where the target row
+ * for an update cannot be found.
+ */
+bool
+FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ bool found;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+
+ if (OidIsValid(localindexoid))
+ indexbitmap = build_index_column_bitmap(localindexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!dead)
+ continue;
+
+ xmax = HeapTupleHeaderGetRawXmax(tuple->t_data);
+ TransactionIdGetCommitTsData(xmax, &localts, &localorigin);
+
+ /* Select the dead tuple with the most recent commit timestamp */
+ if (*delete_time == 0 ||
+ TimestampDifferenceExceeds(*delete_time, localts, 0))
+ {
+ *delete_xid = xmax;
+ *delete_time = localts;
+ *delete_origin = localorigin;
+ found = true;
+ }
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return found;
+}
+
/*
* 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 f4a31ef534..286f5ec257 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -139,6 +139,24 @@
* failover = true when creating the subscription. Enabling failover allows us
* to smoothly transition to the promoted standby, ensuring that we can
* subscribe to the new primary without losing any data.
+ *
+ * CONFLICT DETECTION (update_deleted)
+ * ----------------------
+ * When applying an update operation, if the target row cannot be found, we
+ * scan the table again to locate the most recently deleted row that matches
+ * the old column values of the remote update operation and is not yet
+ * removable by VACUUM.
+ *
+ * To detect the update_deleted conflict stably and correctly in a
+ * bidirectional cluster, we must ensure that dead tuples cannot be cleaned by
+ * VACUUM until the DELETE operations that occurred on the local node have been
+ * replayed on the subscriber. Additionally, any changes on remote nodes that
+ * occurred before the replay of these DELETE operations must also be replayed
+ * locally.
+ *
+ * Therefore, it is necessary to add the replication slot name corresponding to
+ * the publisher (e.g., the slot acquired by the walsender that sends changes
+ * back to the publisher) in the feedback_slots options.
*-------------------------------------------------------------------------
*/
@@ -2672,6 +2690,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2688,15 +2709,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2707,7 +2724,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2726,19 +2743,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(localrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ replica_index, &localxid,
+ &localts, &localorigin))
+ 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3059,7 +3084,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3069,17 +3094,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(partrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ replica_index, &localxid,
+ &localts, &localorigin))
+ 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3091,7 +3124,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3102,7 +3135,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
@@ -4048,8 +4081,8 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* position have been flushed to the remote node corresponding to the feedback
* slots. In this context, the remote node indicates the publisher, as the
* feedback is currently used only in a bidirectional cluster to preserve old
- * row versions for conflict detection purposes (see the comments for
- * ReplicationSlot in slot.h for details).
+ * row versions for conflict detection purposes (see the comments atop worker.c
+ * for details).
*
* If feedback_slots is NULL and a status might have already been sent to
* update the xmin value of the slot, an InvalidXLogRecPtr is sent. This
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 97dc09ac0d..aa4a5fb755 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1966,7 +1966,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -1994,11 +1994,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index ff5436acac..a5a7ee499d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5538,9 +5538,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..bb04ef86c2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index a1626f3fae..569be4ed04 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2143,11 +2143,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v21-0005-Support-copying-xmin-value-of-slots-during-slots.patchapplication/octet-stream; name=v21-0005-Support-copying-xmin-value-of-slots-during-slots.patchDownload
From 5b2bf6411e0792fac0e667d5dee3272ba3eb0908 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 5 Sep 2024 12:41:52 +0800
Subject: [PATCH v21 5/6] Support copying xmin value of slots during slotsync
---
src/backend/replication/logical/slotsync.c | 17 ++++++--
.../t/040_standby_failover_slots_sync.pl | 39 +++++++++++++++----
2 files changed, 45 insertions(+), 11 deletions(-)
diff --git a/src/backend/replication/logical/slotsync.c b/src/backend/replication/logical/slotsync.c
index f9649eec1a..7e8b8767a5 100644
--- a/src/backend/replication/logical/slotsync.c
+++ b/src/backend/replication/logical/slotsync.c
@@ -145,6 +145,9 @@ typedef struct RemoteSlot
/* RS_INVAL_NONE if valid, or the reason of invalidation */
ReplicationSlotInvalidationCause invalidated;
+
+ /* Valid iff the feedback_slot is enabled */
+ XLogRecPtr xmin;
} RemoteSlot;
static void slotsync_failure_callback(int code, Datum arg);
@@ -231,7 +234,8 @@ update_local_synced_slot(RemoteSlot *remote_slot, Oid remote_dbid,
else if (remote_slot->confirmed_lsn > slot->data.confirmed_flush ||
remote_slot->restart_lsn > slot->data.restart_lsn ||
TransactionIdFollows(remote_slot->catalog_xmin,
- slot->data.catalog_xmin))
+ slot->data.catalog_xmin) ||
+ TransactionIdFollows(remote_slot->xmin, slot->data.xmin))
{
/*
* We can't directly copy the remote slot's LSN or xmin unless there
@@ -252,6 +256,7 @@ update_local_synced_slot(RemoteSlot *remote_slot, Oid remote_dbid,
slot->data.restart_lsn = remote_slot->restart_lsn;
slot->data.confirmed_flush = remote_slot->confirmed_lsn;
slot->data.catalog_xmin = remote_slot->catalog_xmin;
+ slot->data.xmin = remote_slot->xmin;
SpinLockRelease(&slot->mutex);
if (found_consistent_snapshot)
@@ -316,6 +321,7 @@ update_local_synced_slot(RemoteSlot *remote_slot, Oid remote_dbid,
{
SpinLockAcquire(&slot->mutex);
slot->effective_catalog_xmin = remote_slot->catalog_xmin;
+ slot->effective_xmin = remote_slot->xmin;
SpinLockRelease(&slot->mutex);
ReplicationSlotsComputeRequiredXmin(false);
@@ -790,9 +796,9 @@ synchronize_one_slot(RemoteSlot *remote_slot, Oid remote_dbid)
static bool
synchronize_slots(WalReceiverConn *wrconn)
{
-#define SLOTSYNC_COLUMN_COUNT 9
+#define SLOTSYNC_COLUMN_COUNT 10
Oid slotRow[SLOTSYNC_COLUMN_COUNT] = {TEXTOID, TEXTOID, LSNOID,
- LSNOID, XIDOID, BOOLOID, BOOLOID, TEXTOID, TEXTOID};
+ LSNOID, XIDOID, BOOLOID, BOOLOID, TEXTOID, TEXTOID, XIDOID};
WalRcvExecResult *res;
TupleTableSlot *tupslot;
@@ -801,7 +807,7 @@ synchronize_slots(WalReceiverConn *wrconn)
bool started_tx = false;
const char *query = "SELECT slot_name, plugin, confirmed_flush_lsn,"
" restart_lsn, catalog_xmin, two_phase, failover,"
- " database, invalidation_reason"
+ " database, invalidation_reason, xmin"
" FROM pg_catalog.pg_replication_slots"
" WHERE failover and NOT temporary";
@@ -867,6 +873,9 @@ synchronize_slots(WalReceiverConn *wrconn)
remote_slot->invalidated = isnull ? RS_INVAL_NONE :
GetSlotInvalidationCause(TextDatumGetCString(d));
+ d = slot_getattr(tupslot, ++col, &isnull);
+ remote_slot->xmin = isnull ? InvalidXLogRecPtr : DatumGetTransactionId(d);
+
/* Sanity check */
Assert(col == SLOTSYNC_COLUMN_COUNT);
diff --git a/src/test/recovery/t/040_standby_failover_slots_sync.pl b/src/test/recovery/t/040_standby_failover_slots_sync.pl
index a474c626d9..f267d0575d 100644
--- a/src/test/recovery/t/040_standby_failover_slots_sync.pl
+++ b/src/test/recovery/t/040_standby_failover_slots_sync.pl
@@ -33,6 +33,7 @@ my $publisher_connstr = $publisher->connstr . ' dbname=postgres';
# Create a subscriber node, wait for sync to complete
my $subscriber1 = PostgreSQL::Test::Cluster->new('subscriber1');
$subscriber1->init;
+$subscriber1->append_conf('postgresql.conf', 'wal_level = logical');
$subscriber1->start;
# Capture the time before the logical failover slot is created on the
@@ -542,8 +543,8 @@ $standby1->reload;
$standby1->wait_for_log(qr/slot sync worker started/, $log_offset);
##################################################
-# Test to confirm that confirmed_flush_lsn of the logical slot on the primary
-# is synced to the standby via the slot sync worker.
+# Test to confirm that confirmed_flush_lsn and xmin of the logical slot on the
+# primary is synced to the standby via the slot sync worker.
##################################################
# Insert data on the primary
@@ -553,11 +554,15 @@ $primary->safe_psql(
INSERT INTO tab_int SELECT generate_series(1, 10);
]);
+# Create another replication slot needed for feedback_slots option
+$subscriber1->safe_psql('postgres',
+ "SELECT pg_create_logical_replication_slot('for_feedback', 'test_decoding');");
+
# Subscribe to the new table data and wait for it to arrive
$subscriber1->safe_psql(
'postgres', qq[
CREATE TABLE tab_int (a int PRIMARY KEY);
- CREATE SUBSCRIPTION regress_mysub1 CONNECTION '$publisher_connstr' PUBLICATION regress_mypub WITH (slot_name = lsub1_slot, failover = true, create_slot = false);
+ CREATE SUBSCRIPTION regress_mysub1 CONNECTION '$publisher_connstr' PUBLICATION regress_mypub WITH (slot_name = lsub1_slot, failover = true, create_slot = false, feedback_slots = 'for_feedback');
]);
$subscriber1->wait_for_subscription_sync;
@@ -573,17 +578,37 @@ $primary->poll_query_until(
"SELECT COUNT(*) FROM pg_catalog.pg_replication_slots WHERE slot_name = 'lsub1_slot' AND active='f'",
1);
-# Get the confirmed_flush_lsn for the logical slot lsub1_slot on the primary
+# Get confirmed_flush_lsn for the logical slot lsub1_slot on the primary
my $primary_flush_lsn = $primary->safe_psql('postgres',
"SELECT confirmed_flush_lsn from pg_replication_slots WHERE slot_name = 'lsub1_slot';"
);
-# Confirm that confirmed_flush_lsn of lsub1_slot slot is synced to the standby
+# Get xmin as well, but ensure the attribute has been updated
+ok( $primary->poll_query_until(
+ 'postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'lsub1_slot';"
+ ),
+ 'xmin is set');
+
+my $primary_xmin = $primary->safe_psql('postgres',
+ "SELECT xmin from pg_replication_slots WHERE slot_name = 'lsub1_slot';"
+);
+
+# Confirm that both confirmed_flush_lsn and xmin of lsub1_slot slot are synced
+# to the standby
ok( $standby1->poll_query_until(
'postgres',
- "SELECT '$primary_flush_lsn' = confirmed_flush_lsn from pg_replication_slots WHERE slot_name = 'lsub1_slot' AND synced AND NOT temporary;"
+ "SELECT '$primary_flush_lsn' = confirmed_flush_lsn AND '$primary_xmin' = xmin from pg_replication_slots WHERE slot_name = 'lsub1_slot' AND synced AND NOT temporary;"
),
- 'confirmed_flush_lsn of slot lsub1_slot synced to standby');
+ 'confirmed_flush_lsn and xmin of slot lsub1_slot synced to standby');
+
+# Reset feedback_slots and drop the slot because it won't be used by upcoming
+# tests
+$subscriber1->safe_psql('postgres',
+ "ALTER SUBSCRIPTION regress_mysub1 SET (feedback_slots = NONE);");
+
+$subscriber1->safe_psql('postgres',
+ "SELECT pg_drop_replication_slot('for_feedback');");
##################################################
# Test that logical failover replication slots wait for the specified
--
2.30.0.windows.2
On Thu, Sep 5, 2024 at 5:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Hi hackers,
I am starting a new thread to discuss and propose the conflict detection for
update_deleted scenarios during logical replication. This conflict occurs when
the apply worker cannot find the target tuple to be updated, as the tuple might
have been removed by another origin.---
BACKGROUND
---Currently, when the apply worker cannot find the target tuple during an update,
an update_missing conflict is logged. However, to facilitate future automatic
conflict resolution, it has been agreed[1][2] that we need to detect both
update_missing and update_deleted conflicts. Specifically, we will detect an
update_deleted conflict if any dead tuple matching the old key value of the
update operation is found; otherwise, it will be classified as update_missing.Detecting both update_deleted and update_missing conflicts is important for
achieving eventual consistency in a bidirectional cluster, because the
resolution for each conflict type can differs. For example, for an
update_missing conflict, a feasible solution might be converting the update to
an insert and applying it. While for an update_deleted conflict, the preferred
approach could be to skip the update or compare the timestamps of the delete
transactions with the remote update transaction's and choose the most recent
one. For additional context, please refer to [3], which gives examples about
how these differences could lead to data divergence.---
ISSUES and SOLUTION
---To detect update_deleted conflicts, we need to search for dead tuples in the
table. However, dead tuples can be removed by VACUUM at any time. Therefore, to
ensure consistent and accurate conflict detection, tuples deleted by other
origins must not be removed by VACUUM before the conflict detection process. If
the tuples are removed prematurely, it might lead to incorrect conflict
identification and resolution, causing data divergence between nodes.Here is an example of how VACUUM could affect conflict detection and how to
prevent this issue. Assume we have a bidirectional cluster with two nodes, A
and B.Node A:
T1: INSERT INTO t (id, value) VALUES (1,1);
T2: DELETE FROM t WHERE id = 1;Node B:
T3: UPDATE t SET value = 2 WHERE id = 1;To retain the deleted tuples, the initial idea was that once transaction T2 had
been applied to both nodes, there was no longer a need to preserve the dead
tuple on Node A. However, a scenario arises where transactions T3 and T2 occur
concurrently, with T3 committing slightly earlier than T2. In this case, if
Node B applies T2 and Node A removes the dead tuple (1,1) via VACUUM, and then
Node A applies T3 after the VACUUM operation, it can only result in an
update_missing conflict. Given that the default resolution for update_missing
conflicts is apply_or_skip (e.g. convert update to insert if possible and apply
the insert), Node A will eventually hold a row (1,2) while Node B becomes
empty, causing data inconsistency.Therefore, the strategy needs to be expanded as follows: Node A cannot remove
the dead tuple until:
(a) The DELETE operation is replayed on all remote nodes, *AND*
(b) The transactions on logical standbys occurring before the replay of Node
A's DELETE are replayed on Node A as well.---
THE DESIGN
---To achieve the above, we plan to allow the logical walsender to maintain and
advance the slot.xmin to protect the data in the user table and introduce a new
logical standby feedback message. This message reports the WAL position that
has been replayed on the logical standby *AND* the changes occurring on the
logical standby before the WAL position are also replayed to the walsender's
node (where the walsender is running). After receiving the new feedback
message, the walsender will advance the slot.xmin based on the flush info,
similar to the advancement of catalog_xmin. Currently, the effective_xmin/xmin
of logical slot are unused during logical replication, so I think it's safe and
won't cause side-effect to reuse the xmin for this feature.We have introduced a new subscription option (feedback_slots='slot1,...'),
where these slots will be used to check condition (b): the transactions on
logical standbys occurring before the replay of Node A's DELETE are replayed on
Node A as well. Therefore, on Node B, users should specify the slots
corresponding to Node A in this option. The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a feedback
message to the walsender. -- I also thought of making it an automaic way, e.g.
let apply worker select the slots that acquired by the walsenders which connect
to the same remote server(e.g. if apply worker's connection info or some other
flags is same as the walsender's connection info). But it seems tricky because
if some slots are inactive which means the walsenders are not there, the apply
worker could not find the correct slots to check unless we save the host along
with the slot's persistence data.The new feedback message is sent only if feedback_slots is not NULL. If the
slots in feedback_slots are removed, a final message containing
InvalidXLogRecPtr will be sent to inform the walsender to forget about the
slot.xmin.To detect update_deleted conflicts during update operations, if the target row
cannot be found, 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.Please refer to the attached POC patch set which implements above design. The
patch set is split into some parts to make it easier for the initial review.
Please note that each patch is interdependent and cannot work independently.Thanks a lot to Kuroda-San and Amit for the off-list discussion.
Suggestions and comments are highly appreciated !
Thank You Hou-San for explaining the design. But to make it easier to
understand, would you be able to explain the sequence/timeline of the
*new* actions performed by the walsender and the apply processes for
the given example along with new feedback_slot config needed
Node A: (Procs: walsenderA, applyA)
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AM
Node B: (Procs: walsenderB, applyB)
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AM
thanks
Shveta
On Tuesday, September 10, 2024 2:45 PM shveta malik <shveta.malik@gmail.com> wrote:
---
THE DESIGN
---To achieve the above, we plan to allow the logical walsender to
maintain and advance the slot.xmin to protect the data in the user
table and introduce a new logical standby feedback message. This
message reports the WAL position that has been replayed on the logical
standby *AND* the changes occurring on the logical standby before the
WAL position are also replayed to the walsender's node (where the
walsender is running). After receiving the new feedback message, the
walsender will advance the slot.xmin based on the flush info, similar
to the advancement of catalog_xmin. Currently, the effective_xmin/xmin
of logical slot are unused during logical replication, so I think it's safe andwon't cause side-effect to reuse the xmin for this feature.
We have introduced a new subscription option
(feedback_slots='slot1,...'), where these slots will be used to check
condition (b): the transactions on logical standbys occurring before
the replay of Node A's DELETE are replayed on Node A as well.
Therefore, on Node B, users should specify the slots corresponding to
Node A in this option. The apply worker will get the oldest confirmed
flush LSN among the specified slots and send the LSN as a feedbackmessage to the walsender. -- I also thought of making it an automaic way, e.g.
let apply worker select the slots that acquired by the walsenders
which connect to the same remote server(e.g. if apply worker's
connection info or some other flags is same as the walsender's
connection info). But it seems tricky because if some slots are
inactive which means the walsenders are not there, the apply worker
could not find the correct slots to check unless we save the host along withthe slot's persistence data.
The new feedback message is sent only if feedback_slots is not NULL.
If the slots in feedback_slots are removed, a final message containing
InvalidXLogRecPtr will be sent to inform the walsender to forget about
the slot.xmin.To detect update_deleted conflicts during update operations, if the
target row cannot be found, we perform an additional scan of the table usingsnapshotAny.
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 informationthat deleted the tuple.
Please refer to the attached POC patch set which implements above
design. The patch set is split into some parts to make it easier for the initialreview.
Please note that each patch is interdependent and cannot work
independently.
Thanks a lot to Kuroda-San and Amit for the off-list discussion.
Suggestions and comments are highly appreciated !
Thank You Hou-San for explaining the design. But to make it easier to
understand, would you be able to explain the sequence/timeline of the
*new* actions performed by the walsender and the apply processes for the
given example along with new feedback_slot config neededNode A: (Procs: walsenderA, applyA)
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AMNode B: (Procs: walsenderB, applyB)
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AM
Thanks for reviewing! Let me elaborate further on the example:
On node A, feedback_slots should include the logical slot that used to replicate changes
from Node A to Node B. On node B, feedback_slots should include the logical
slot that replicate changes from Node B to Node A.
Assume the slot.xmin on Node A has been initialized to a valid number(740) before the
following flow:
Node A executed T1 - 10.00 AM
T1 replicated and applied on Node B - 10.0001 AM
Node B executed T3 - 10.01 AM
Node A executed T2 (741) - 10.02 AM
T2 replicated and applied on Node B (delete_missing) - 10.03 AM
T3 replicated and applied on Node A (new action, detect update_deleted) - 10.04 AM
(new action) Apply worker on Node B has confirmed that T2 has been applied
locally and the transactions before T2 (e.g., T3) has been replicated and
applied to Node A (e.g. feedback_slot.confirmed_flush_lsn >= lsn of the local
replayed T2), thus send the new feedback message to Node A. - 10.05 AM
(new action) Walsender on Node A received the message and would advance the slot.xmin.- 10.06 AM
Then, after the slot.xmin is advanced to a number greater than 741, the VACUUM would be able to
remove the dead tuple on Node A.
Best Regards,
Hou zj
On Tue, Sep 10, 2024 at 1:40 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 10, 2024 2:45 PM shveta malik <shveta.malik@gmail.com> wrote:
---
THE DESIGN
---To achieve the above, we plan to allow the logical walsender to
maintain and advance the slot.xmin to protect the data in the user
table and introduce a new logical standby feedback message. This
message reports the WAL position that has been replayed on the logical
standby *AND* the changes occurring on the logical standby before the
WAL position are also replayed to the walsender's node (where the
walsender is running). After receiving the new feedback message, the
walsender will advance the slot.xmin based on the flush info, similar
to the advancement of catalog_xmin. Currently, the effective_xmin/xmin
of logical slot are unused during logical replication, so I think it's safe andwon't cause side-effect to reuse the xmin for this feature.
We have introduced a new subscription option
(feedback_slots='slot1,...'), where these slots will be used to check
condition (b): the transactions on logical standbys occurring before
the replay of Node A's DELETE are replayed on Node A as well.
Therefore, on Node B, users should specify the slots corresponding to
Node A in this option. The apply worker will get the oldest confirmed
flush LSN among the specified slots and send the LSN as a feedbackmessage to the walsender. -- I also thought of making it an automaic way, e.g.
let apply worker select the slots that acquired by the walsenders
which connect to the same remote server(e.g. if apply worker's
connection info or some other flags is same as the walsender's
connection info). But it seems tricky because if some slots are
inactive which means the walsenders are not there, the apply worker
could not find the correct slots to check unless we save the host along withthe slot's persistence data.
The new feedback message is sent only if feedback_slots is not NULL.
If the slots in feedback_slots are removed, a final message containing
InvalidXLogRecPtr will be sent to inform the walsender to forget about
the slot.xmin.To detect update_deleted conflicts during update operations, if the
target row cannot be found, we perform an additional scan of the table usingsnapshotAny.
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 informationthat deleted the tuple.
Please refer to the attached POC patch set which implements above
design. The patch set is split into some parts to make it easier for the initialreview.
Please note that each patch is interdependent and cannot work
independently.
Thanks a lot to Kuroda-San and Amit for the off-list discussion.
Suggestions and comments are highly appreciated !
Thank You Hou-San for explaining the design. But to make it easier to
understand, would you be able to explain the sequence/timeline of the
*new* actions performed by the walsender and the apply processes for the
given example along with new feedback_slot config neededNode A: (Procs: walsenderA, applyA)
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AMNode B: (Procs: walsenderB, applyB)
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AMThanks for reviewing! Let me elaborate further on the example:
On node A, feedback_slots should include the logical slot that used to replicate changes
from Node A to Node B. On node B, feedback_slots should include the logical
slot that replicate changes from Node B to Node A.Assume the slot.xmin on Node A has been initialized to a valid number(740) before the
following flow:Node A executed T1 - 10.00 AM
T1 replicated and applied on Node B - 10.0001 AM
Node B executed T3 - 10.01 AM
Node A executed T2 (741) - 10.02 AM
T2 replicated and applied on Node B (delete_missing) - 10.03 AM
Not related to this feature, but do you mean delete_origin_differ here?
T3 replicated and applied on Node A (new action, detect update_deleted) - 10.04 AM
(new action) Apply worker on Node B has confirmed that T2 has been applied
locally and the transactions before T2 (e.g., T3) has been replicated and
applied to Node A (e.g. feedback_slot.confirmed_flush_lsn >= lsn of the local
replayed T2), thus send the new feedback message to Node A. - 10.05 AM(new action) Walsender on Node A received the message and would advance the slot.xmin.- 10.06 AM
Then, after the slot.xmin is advanced to a number greater than 741, the VACUUM would be able to
remove the dead tuple on Node A.
Thanks for the example. Can you please review below and let me know if
my understanding is correct.
1)
In a bidirectional replication setup, the user has to create slots in
a way that NodeA's sub's slot is Node B's feedback_slot and Node B's
sub's slot is Node A's feedback slot. And then only this feature will
work well, is it correct to say?
2)
Now coming back to multiple feedback_slots in a subscription, is the
below correct:
Say Node A has publications and subscriptions as follow:
------------------
A_pub1
A_sub1 (subscribing to B_pub1 with the default slot_name of A_sub1)
A_sub2 (subscribing to B_pub2 with the default slot_name of A_sub2)
A_sub3 (subscribing to B_pub3 with the default slot_name of A_sub3)
Say Node B has publications and subscriptions as follow:
------------------
B_sub1 (subscribing to A_pub1 with the default slot_name of B_sub1)
B_pub1
B_pub2
B_pub3
Then what will be the feedback_slot configuration for all
subscriptions of A and B? Is below correct:
------------------
A_sub1, A_sub2, A_sub3: feedback_slots=B_sub1
B_sub1: feedback_slots=A_sub1,A_sub2, A_sub3
3)
If the above is true, then do we have a way to make sure that the user
has given this configuration exactly the above way? If users end up
giving feedback_slots as some random slot (say A_slot4 or incomplete
list), do we validate that? (I have not looked at code yet, just
trying to understand design first).
4)
Now coming to this:
The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a feedback
message to the walsender.
There will be one apply worker on B which will be due to B_sub1, so
will it check confirmed_lsn of all slots A_sub1,A_sub2, A_sub3? Won't
it be sufficient to check confimed_lsn of say slot A_sub1 alone which
has subscribed to table 't' on which delete has been performed? Rest
of the lots (A_sub2, A_sub3) might have subscribed to different
tables?
thanks
Shveta
On Tuesday, September 10, 2024 5:56 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Sep 10, 2024 at 1:40 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Tuesday, September 10, 2024 2:45 PM shveta malik
<shveta.malik@gmail.com> wrote:
Thank You Hou-San for explaining the design. But to make it easier
to understand, would you be able to explain the sequence/timeline of
the
*new* actions performed by the walsender and the apply processes for
the given example along with new feedback_slot config neededNode A: (Procs: walsenderA, applyA)
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AMNode B: (Procs: walsenderB, applyB)
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AMThanks for reviewing! Let me elaborate further on the example:
On node A, feedback_slots should include the logical slot that used to
replicate changes from Node A to Node B. On node B, feedback_slots
should include the logical slot that replicate changes from Node B to Node A.Assume the slot.xmin on Node A has been initialized to a valid
number(740) before the following flow:Node A executed T1 - 10.00 AM
T1 replicated and applied on Node B - 10.0001 AM
Node B executed T3 - 10.01 AM
Node A executed T2 (741) - 10.02 AM
T2 replicated and applied on Node B (delete_missing) - 10.03 AMNot related to this feature, but do you mean delete_origin_differ here?
Oh sorry, It's a miss. I meant delete_origin_differ.
T3 replicated and applied on Node A (new action, detect
update_deleted) - 10.04 AM
(new action) Apply worker on Node B has confirmed that T2 has been
applied locally and the transactions before T2 (e.g., T3) has been
replicated and applied to Node A (e.g. feedback_slot.confirmed_flush_lsn
= lsn of the local
replayed T2), thus send the new feedback message to Node A.- 10.05 AM
(new action) Walsender on Node A received the message and would
advance the slot.xmin.- 10.06 AMThen, after the slot.xmin is advanced to a number greater than 741,
the VACUUM would be able to remove the dead tuple on Node A.Thanks for the example. Can you please review below and let me know if my
understanding is correct.1)
In a bidirectional replication setup, the user has to create slots in a way that
NodeA's sub's slot is Node B's feedback_slot and Node B's sub's slot is Node
A's feedback slot. And then only this feature will work well, is it correct to say?
Yes, your understanding is correct.
2)
Now coming back to multiple feedback_slots in a subscription, is the below
correct:Say Node A has publications and subscriptions as follow:
------------------
A_pub1A_sub1 (subscribing to B_pub1 with the default slot_name of A_sub1)
A_sub2 (subscribing to B_pub2 with the default slot_name of A_sub2)
A_sub3 (subscribing to B_pub3 with the default slot_name of A_sub3)Say Node B has publications and subscriptions as follow:
------------------
B_sub1 (subscribing to A_pub1 with the default slot_name of B_sub1)B_pub1
B_pub2
B_pub3Then what will be the feedback_slot configuration for all subscriptions of A and
B? Is below correct:
------------------
A_sub1, A_sub2, A_sub3: feedback_slots=B_sub1
B_sub1: feedback_slots=A_sub1,A_sub2, A_sub3
Right. The above configurations are correct.
3)
If the above is true, then do we have a way to make sure that the user has
given this configuration exactly the above way? If users end up giving
feedback_slots as some random slot (say A_slot4 or incomplete list), do we
validate that? (I have not looked at code yet, just trying to understand design
first).
The patch doesn't validate if the feedback slots belong to the correct
subscriptions on remote server. It only validates if the slot is an existing,
valid, logical slot. I think there are few challenges to validate it further.
E.g. We need a way to identify the which server the slot is replicating
changes to, which could be tricky as the slot currently doesn't have any info
to identify the remote server. Besides, the slot could be inactive temporarily
due to some subscriber side error, in which case we cannot verify the
subscription that used it.
4)
Now coming to this:The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a
feedback message to the walsender.There will be one apply worker on B which will be due to B_sub1, so will it
check confirmed_lsn of all slots A_sub1,A_sub2, A_sub3? Won't it be
sufficient to check confimed_lsn of say slot A_sub1 alone which has
subscribed to table 't' on which delete has been performed? Rest of the lots
(A_sub2, A_sub3) might have subscribed to different tables?
I think it's theoretically correct to only check the A_sub1. We could document
that user can do this by identifying the tables that each subscription
replicates, but it may not be user friendly.
Best Regards,
Hou zj
On Thu, Sep 5, 2024 at 5:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
---
ISSUES and SOLUTION
---To detect update_deleted conflicts, we need to search for dead tuples in the
table. However, dead tuples can be removed by VACUUM at any time. Therefore, to
ensure consistent and accurate conflict detection, tuples deleted by other
origins must not be removed by VACUUM before the conflict detection process. If
the tuples are removed prematurely, it might lead to incorrect conflict
identification and resolution, causing data divergence between nodes.Here is an example of how VACUUM could affect conflict detection and how to
prevent this issue. Assume we have a bidirectional cluster with two nodes, A
and B.Node A:
T1: INSERT INTO t (id, value) VALUES (1,1);
T2: DELETE FROM t WHERE id = 1;Node B:
T3: UPDATE t SET value = 2 WHERE id = 1;To retain the deleted tuples, the initial idea was that once transaction T2 had
been applied to both nodes, there was no longer a need to preserve the dead
tuple on Node A. However, a scenario arises where transactions T3 and T2 occur
concurrently, with T3 committing slightly earlier than T2. In this case, if
Node B applies T2 and Node A removes the dead tuple (1,1) via VACUUM, and then
Node A applies T3 after the VACUUM operation, it can only result in an
update_missing conflict. Given that the default resolution for update_missing
conflicts is apply_or_skip (e.g. convert update to insert if possible and apply
the insert), Node A will eventually hold a row (1,2) while Node B becomes
empty, causing data inconsistency.Therefore, the strategy needs to be expanded as follows: Node A cannot remove
the dead tuple until:
(a) The DELETE operation is replayed on all remote nodes, *AND*
(b) The transactions on logical standbys occurring before the replay of Node
A's DELETE are replayed on Node A as well.---
THE DESIGN
---To achieve the above, we plan to allow the logical walsender to maintain and
advance the slot.xmin to protect the data in the user table and introduce a new
logical standby feedback message. This message reports the WAL position that
has been replayed on the logical standby *AND* the changes occurring on the
logical standby before the WAL position are also replayed to the walsender's
node (where the walsender is running). After receiving the new feedback
message, the walsender will advance the slot.xmin based on the flush info,
similar to the advancement of catalog_xmin. Currently, the effective_xmin/xmin
of logical slot are unused during logical replication, so I think it's safe and
won't cause side-effect to reuse the xmin for this feature.We have introduced a new subscription option (feedback_slots='slot1,...'),
where these slots will be used to check condition (b): the transactions on
logical standbys occurring before the replay of Node A's DELETE are replayed on
Node A as well. Therefore, on Node B, users should specify the slots
corresponding to Node A in this option. The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a feedback
message to the walsender. -- I also thought of making it an automaic way, e.g.
let apply worker select the slots that acquired by the walsenders which connect
to the same remote server(e.g. if apply worker's connection info or some other
flags is same as the walsender's connection info). But it seems tricky because
if some slots are inactive which means the walsenders are not there, the apply
worker could not find the correct slots to check unless we save the host along
with the slot's persistence data.The new feedback message is sent only if feedback_slots is not NULL.
Don't you need to deal with versioning stuff for sending this new
message? I mean what if the receiver side of this message is old and
doesn't support this new message.
One minor comment on 0003
=======================
1.
get_slot_confirmed_flush()
{
...
+ /*
+ * To prevent concurrent slot dropping and creation while filtering the
+ * slots, take the ReplicationSlotControlLock outside of the loop.
+ */
+ LWLockAcquire(ReplicationSlotControlLock, LW_SHARED);
+
+ foreach_ptr(String, name, MySubscription->feedback_slots)
+ {
+ XLogRecPtr confirmed_flush;
+ ReplicationSlot *slot;
+
+ slot = ValidateAndGetFeedbackSlot(strVal(name));
Why do we need to validate slots each time here? Isn't it better to do it once?
--
With Regards,
Amit Kapila.
On Tuesday, September 10, 2024 7:25 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Sep 5, 2024 at 5:07 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:---
ISSUES and SOLUTION
---To detect update_deleted conflicts, we need to search for dead tuples
in the table. However, dead tuples can be removed by VACUUM at any
time. Therefore, to ensure consistent and accurate conflict detection,
tuples deleted by other origins must not be removed by VACUUM before
the conflict detection process. If the tuples are removed prematurely,
it might lead to incorrect conflict identification and resolution, causing datadivergence between nodes.
Here is an example of how VACUUM could affect conflict detection and
how to prevent this issue. Assume we have a bidirectional cluster with
two nodes, A and B.Node A:
T1: INSERT INTO t (id, value) VALUES (1,1);
T2: DELETE FROM t WHERE id = 1;Node B:
T3: UPDATE t SET value = 2 WHERE id = 1;To retain the deleted tuples, the initial idea was that once
transaction T2 had been applied to both nodes, there was no longer a
need to preserve the dead tuple on Node A. However, a scenario arises
where transactions T3 and T2 occur concurrently, with T3 committing
slightly earlier than T2. In this case, if Node B applies T2 and Node
A removes the dead tuple (1,1) via VACUUM, and then Node A applies T3
after the VACUUM operation, it can only result in an update_missing
conflict. Given that the default resolution for update_missing
conflicts is apply_or_skip (e.g. convert update to insert if possible
and apply the insert), Node A will eventually hold a row (1,2) while Node Bbecomes empty, causing data inconsistency.
Therefore, the strategy needs to be expanded as follows: Node A cannot
remove the dead tuple until:
(a) The DELETE operation is replayed on all remote nodes, *AND*
(b) The transactions on logical standbys occurring before the replay
of Node A's DELETE are replayed on Node A as well.---
THE DESIGN
---To achieve the above, we plan to allow the logical walsender to
maintain and advance the slot.xmin to protect the data in the user
table and introduce a new logical standby feedback message. This
message reports the WAL position that has been replayed on the logical
standby *AND* the changes occurring on the logical standby before the
WAL position are also replayed to the walsender's node (where the
walsender is running). After receiving the new feedback message, the
walsender will advance the slot.xmin based on the flush info, similar
to the advancement of catalog_xmin. Currently, the effective_xmin/xmin
of logical slot are unused during logical replication, so I think it's safe andwon't cause side-effect to reuse the xmin for this feature.
We have introduced a new subscription option
(feedback_slots='slot1,...'), where these slots will be used to check
condition (b): the transactions on logical standbys occurring before
the replay of Node A's DELETE are replayed on Node A as well.
Therefore, on Node B, users should specify the slots corresponding to
Node A in this option. The apply worker will get the oldest confirmed
flush LSN among the specified slots and send the LSN as a feedbackmessage to the walsender. -- I also thought of making it an automaic way, e.g.
let apply worker select the slots that acquired by the walsenders
which connect to the same remote server(e.g. if apply worker's
connection info or some other flags is same as the walsender's
connection info). But it seems tricky because if some slots are
inactive which means the walsenders are not there, the apply worker
could not find the correct slots to check unless we save the host along withthe slot's persistence data.
The new feedback message is sent only if feedback_slots is not NULL.
Don't you need to deal with versioning stuff for sending this new message? I
mean what if the receiver side of this message is old and doesn't support this
new message.
Yes, I think we can avoid sending the new message if the remote server version
doesn't support handling this message (e.g. server_version < 18). Will address
this in next version.
One minor comment on 0003 ======================= 1. get_slot_confirmed_flush() { ... + /* + * To prevent concurrent slot dropping and creation while filtering the + * slots, take the ReplicationSlotControlLock outside of the loop. + */ + LWLockAcquire(ReplicationSlotControlLock, LW_SHARED); + + foreach_ptr(String, name, MySubscription->feedback_slots) { XLogRecPtr + confirmed_flush; ReplicationSlot *slot; + + slot = ValidateAndGetFeedbackSlot(strVal(name));Why do we need to validate slots each time here? Isn't it better to do it once?
I think it's possible that the slot was correct but changed or dropped later,
so it could be useful to give a warning in this case to hint user to adjust the
slots, otherwise, the xmin of the publisher's slot won't be advanced and might
cause dead tuples accumulation. This is similar to the checks we performed for
the slots in "synchronized_standby_slots". (E.g. StandbySlotsHaveCaughtup)
Best Regards,
Hou zj
On Tue, Sep 10, 2024 at 4:30 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 10, 2024 5:56 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Sep 10, 2024 at 1:40 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Tuesday, September 10, 2024 2:45 PM shveta malik
<shveta.malik@gmail.com> wrote:
Thank You Hou-San for explaining the design. But to make it easier
to understand, would you be able to explain the sequence/timeline of
the
*new* actions performed by the walsender and the apply processes for
the given example along with new feedback_slot config neededNode A: (Procs: walsenderA, applyA)
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AMNode B: (Procs: walsenderB, applyB)
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AMThanks for reviewing! Let me elaborate further on the example:
On node A, feedback_slots should include the logical slot that used to
replicate changes from Node A to Node B. On node B, feedback_slots
should include the logical slot that replicate changes from Node B to Node A.Assume the slot.xmin on Node A has been initialized to a valid
number(740) before the following flow:Node A executed T1 - 10.00 AM
T1 replicated and applied on Node B - 10.0001 AM
Node B executed T3 - 10.01 AM
Node A executed T2 (741) - 10.02 AM
T2 replicated and applied on Node B (delete_missing) - 10.03 AMNot related to this feature, but do you mean delete_origin_differ here?
Oh sorry, It's a miss. I meant delete_origin_differ.
T3 replicated and applied on Node A (new action, detect
update_deleted) - 10.04 AM
(new action) Apply worker on Node B has confirmed that T2 has been
applied locally and the transactions before T2 (e.g., T3) has been
replicated and applied to Node A (e.g. feedback_slot.confirmed_flush_lsn
= lsn of the local
replayed T2), thus send the new feedback message to Node A.- 10.05 AM
(new action) Walsender on Node A received the message and would
advance the slot.xmin.- 10.06 AMThen, after the slot.xmin is advanced to a number greater than 741,
the VACUUM would be able to remove the dead tuple on Node A.Thanks for the example. Can you please review below and let me know if my
understanding is correct.1)
In a bidirectional replication setup, the user has to create slots in a way that
NodeA's sub's slot is Node B's feedback_slot and Node B's sub's slot is Node
A's feedback slot. And then only this feature will work well, is it correct to say?Yes, your understanding is correct.
2)
Now coming back to multiple feedback_slots in a subscription, is the below
correct:Say Node A has publications and subscriptions as follow:
------------------
A_pub1A_sub1 (subscribing to B_pub1 with the default slot_name of A_sub1)
A_sub2 (subscribing to B_pub2 with the default slot_name of A_sub2)
A_sub3 (subscribing to B_pub3 with the default slot_name of A_sub3)Say Node B has publications and subscriptions as follow:
------------------
B_sub1 (subscribing to A_pub1 with the default slot_name of B_sub1)B_pub1
B_pub2
B_pub3Then what will be the feedback_slot configuration for all subscriptions of A and
B? Is below correct:
------------------
A_sub1, A_sub2, A_sub3: feedback_slots=B_sub1
B_sub1: feedback_slots=A_sub1,A_sub2, A_sub3Right. The above configurations are correct.
Okay. It seems difficult to understand configuration from user's perspective.
3)
If the above is true, then do we have a way to make sure that the user has
given this configuration exactly the above way? If users end up giving
feedback_slots as some random slot (say A_slot4 or incomplete list), do we
validate that? (I have not looked at code yet, just trying to understand design
first).The patch doesn't validate if the feedback slots belong to the correct
subscriptions on remote server. It only validates if the slot is an existing,
valid, logical slot. I think there are few challenges to validate it further.
E.g. We need a way to identify the which server the slot is replicating
changes to, which could be tricky as the slot currently doesn't have any info
to identify the remote server. Besides, the slot could be inactive temporarily
due to some subscriber side error, in which case we cannot verify the
subscription that used it.
Okay, I understand the challenges here.
4)
Now coming to this:The apply worker will get the oldest
confirmed flush LSN among the specified slots and send the LSN as a
feedback message to the walsender.There will be one apply worker on B which will be due to B_sub1, so will it
check confirmed_lsn of all slots A_sub1,A_sub2, A_sub3? Won't it be
sufficient to check confimed_lsn of say slot A_sub1 alone which has
subscribed to table 't' on which delete has been performed? Rest of the lots
(A_sub2, A_sub3) might have subscribed to different tables?I think it's theoretically correct to only check the A_sub1. We could document
that user can do this by identifying the tables that each subscription
replicates, but it may not be user friendly.
Sorry, I fail to understand how user can identify the tables and give
feedback_slots accordingly? I thought feedback_slots is a one time
configuration when replication is setup (or say setup changes in
future); it can not keep on changing with each query. Or am I missing
something?
IMO, it is something which should be identified internally. Since the
query is on table 't1', feedback-slot which is for 't1' shall be used
to check lsn. But on rethinking,this optimization may not be worth the
effort, the identification part could be tricky, so it might be okay
to check all the slots.
~~
Another query is about 3 node setup. I couldn't figure out what would
be feedback_slots setting when it is not bidirectional, as in consider
the case where there are three nodes A,B,C. Node C is subscribing to
both Node A and Node B. Node A and Node B are the ones doing
concurrent "update" and "delete" which will both be replicated to Node
C. In this case what will be the feedback_slots setting on Node C? We
don't have any slots here which will be replicating changes from Node
C to Node A and Node C to Node B. This is given in [3] in your first
email ([1]/messages/by-id/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com)
[1]: /messages/by-id/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
/messages/by-id/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
thanks
Shveta
On Wednesday, September 11, 2024 12:18 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Sep 10, 2024 at 4:30 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Tuesday, September 10, 2024 5:56 PM shveta malik
<shveta.malik@gmail.com> wrote:
Thanks for the example. Can you please review below and let me know
if my understanding is correct.1)
In a bidirectional replication setup, the user has to create slots
in a way that NodeA's sub's slot is Node B's feedback_slot and Node
B's sub's slot is Node A's feedback slot. And then only this feature willwork well, is it correct to say?
Yes, your understanding is correct.
2)
Now coming back to multiple feedback_slots in a subscription, is the
below
correct:Say Node A has publications and subscriptions as follow:
------------------
A_pub1A_sub1 (subscribing to B_pub1 with the default slot_name of A_sub1)
A_sub2 (subscribing to B_pub2 with the default slot_name of A_sub2)
A_sub3 (subscribing to B_pub3 with the default slot_name of A_sub3)Say Node B has publications and subscriptions as follow:
------------------
B_sub1 (subscribing to A_pub1 with the default slot_name of B_sub1)B_pub1
B_pub2
B_pub3Then what will be the feedback_slot configuration for all
subscriptions of A and B? Is below correct:
------------------
A_sub1, A_sub2, A_sub3: feedback_slots=B_sub1
B_sub1: feedback_slots=A_sub1,A_sub2, A_sub3Right. The above configurations are correct.
Okay. It seems difficult to understand configuration from user's perspective.
Right. I think we could give an example in the document to make it clear.
3)
If the above is true, then do we have a way to make sure that the
user has given this configuration exactly the above way? If users
end up giving feedback_slots as some random slot (say A_slot4 or
incomplete list), do we validate that? (I have not looked at code
yet, just trying to understand design first).The patch doesn't validate if the feedback slots belong to the correct
subscriptions on remote server. It only validates if the slot is an
existing, valid, logical slot. I think there are few challenges to validate itfurther.
E.g. We need a way to identify the which server the slot is
replicating changes to, which could be tricky as the slot currently
doesn't have any info to identify the remote server. Besides, the slot
could be inactive temporarily due to some subscriber side error, in
which case we cannot verify the subscription that used it.Okay, I understand the challenges here.
4)
Now coming to this:The apply worker will get the oldest confirmed flush LSN among the
specified slots and send the LSN as a feedback message to the
walsender.There will be one apply worker on B which will be due to B_sub1, so
will it check confirmed_lsn of all slots A_sub1,A_sub2, A_sub3?
Won't it be sufficient to check confimed_lsn of say slot A_sub1
alone which has subscribed to table 't' on which delete has been
performed? Rest of the lots (A_sub2, A_sub3) might have subscribed todifferent tables?
I think it's theoretically correct to only check the A_sub1. We could
document that user can do this by identifying the tables that each
subscription replicates, but it may not be user friendly.Sorry, I fail to understand how user can identify the tables and give
feedback_slots accordingly? I thought feedback_slots is a one time
configuration when replication is setup (or say setup changes in future); it can
not keep on changing with each query. Or am I missing something?
I meant that user have all the publication information(including the tables
added in a publication) that the subscription subscribes to, and could also
have the slot_name, so I think it's possible to identify the tables that each
subscription includes and add the feedback_slots correspondingly before
starting the replication. It would be pretty complicate although possible, so I
prefer to not mention it in the first place if it could not bring much
benefits.
IMO, it is something which should be identified internally. Since the query is on
table 't1', feedback-slot which is for 't1' shall be used to check lsn. But on
rethinking,this optimization may not be worth the effort, the identification part
could be tricky, so it might be okay to check all the slots.
I agree that identifying these internally would add complexity.
~~
Another query is about 3 node setup. I couldn't figure out what would be
feedback_slots setting when it is not bidirectional, as in consider the case
where there are three nodes A,B,C. Node C is subscribing to both Node A and
Node B. Node A and Node B are the ones doing concurrent "update" and
"delete" which will both be replicated to Node C. In this case what will be the
feedback_slots setting on Node C? We don't have any slots here which will be
replicating changes from Node C to Node A and Node C to Node B. This is given
in [3] in your first email ([1])
Thanks for pointing this, the link was a bit misleading. I think the solution
proposed in this thread is only used to allow detecting update_deleted reliably
in a bidirectional cluster. For non- bidirectional cases, it would be more
tricky to predict the timing till when should we retain the dead tuples.
[1]:
/messages/by-id/OS0PR01MB5716BE80DAEB0EE2A
6A5D1F5949D2%40OS0PR01MB5716.jpnprd01.prod.outlook.com
Best Regards,
Hou zj
On Wed, Sep 11, 2024 at 10:15 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wednesday, September 11, 2024 12:18 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Sep 10, 2024 at 4:30 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Tuesday, September 10, 2024 5:56 PM shveta malik
<shveta.malik@gmail.com> wrote:
Thanks for the example. Can you please review below and let me know
if my understanding is correct.1)
In a bidirectional replication setup, the user has to create slots
in a way that NodeA's sub's slot is Node B's feedback_slot and Node
B's sub's slot is Node A's feedback slot. And then only this feature willwork well, is it correct to say?
Yes, your understanding is correct.
2)
Now coming back to multiple feedback_slots in a subscription, is the
below
correct:Say Node A has publications and subscriptions as follow:
------------------
A_pub1A_sub1 (subscribing to B_pub1 with the default slot_name of A_sub1)
A_sub2 (subscribing to B_pub2 with the default slot_name of A_sub2)
A_sub3 (subscribing to B_pub3 with the default slot_name of A_sub3)Say Node B has publications and subscriptions as follow:
------------------
B_sub1 (subscribing to A_pub1 with the default slot_name of B_sub1)B_pub1
B_pub2
B_pub3Then what will be the feedback_slot configuration for all
subscriptions of A and B? Is below correct:
------------------
A_sub1, A_sub2, A_sub3: feedback_slots=B_sub1
B_sub1: feedback_slots=A_sub1,A_sub2, A_sub3Right. The above configurations are correct.
Okay. It seems difficult to understand configuration from user's perspective.
Right. I think we could give an example in the document to make it clear.
3)
If the above is true, then do we have a way to make sure that the
user has given this configuration exactly the above way? If users
end up giving feedback_slots as some random slot (say A_slot4 or
incomplete list), do we validate that? (I have not looked at code
yet, just trying to understand design first).The patch doesn't validate if the feedback slots belong to the correct
subscriptions on remote server. It only validates if the slot is an
existing, valid, logical slot. I think there are few challenges to validate itfurther.
E.g. We need a way to identify the which server the slot is
replicating changes to, which could be tricky as the slot currently
doesn't have any info to identify the remote server. Besides, the slot
could be inactive temporarily due to some subscriber side error, in
which case we cannot verify the subscription that used it.Okay, I understand the challenges here.
4)
Now coming to this:The apply worker will get the oldest confirmed flush LSN among the
specified slots and send the LSN as a feedback message to the
walsender.There will be one apply worker on B which will be due to B_sub1, so
will it check confirmed_lsn of all slots A_sub1,A_sub2, A_sub3?
Won't it be sufficient to check confimed_lsn of say slot A_sub1
alone which has subscribed to table 't' on which delete has been
performed? Rest of the lots (A_sub2, A_sub3) might have subscribed todifferent tables?
I think it's theoretically correct to only check the A_sub1. We could
document that user can do this by identifying the tables that each
subscription replicates, but it may not be user friendly.Sorry, I fail to understand how user can identify the tables and give
feedback_slots accordingly? I thought feedback_slots is a one time
configuration when replication is setup (or say setup changes in future); it can
not keep on changing with each query. Or am I missing something?I meant that user have all the publication information(including the tables
added in a publication) that the subscription subscribes to, and could also
have the slot_name, so I think it's possible to identify the tables that each
subscription includes and add the feedback_slots correspondingly before
starting the replication. It would be pretty complicate although possible, so I
prefer to not mention it in the first place if it could not bring much
benefits.IMO, it is something which should be identified internally. Since the query is on
table 't1', feedback-slot which is for 't1' shall be used to check lsn. But on
rethinking,this optimization may not be worth the effort, the identification part
could be tricky, so it might be okay to check all the slots.I agree that identifying these internally would add complexity.
~~
Another query is about 3 node setup. I couldn't figure out what would be
feedback_slots setting when it is not bidirectional, as in consider the case
where there are three nodes A,B,C. Node C is subscribing to both Node A and
Node B. Node A and Node B are the ones doing concurrent "update" and
"delete" which will both be replicated to Node C. In this case what will be the
feedback_slots setting on Node C? We don't have any slots here which will be
replicating changes from Node C to Node A and Node C to Node B. This is given
in [3] in your first email ([1])Thanks for pointing this, the link was a bit misleading. I think the solution
proposed in this thread is only used to allow detecting update_deleted reliably
in a bidirectional cluster. For non- bidirectional cases, it would be more
tricky to predict the timing till when should we retain the dead tuples.
So in brief, this solution is only for bidrectional setup? For
non-bidirectional, feedback_slots is non-configurable and thus
irrelevant.
Irrespective of above, if user ends up setting feedback_slot to some
random but existing slot which is not at all consuming changes, then
it may so happen that the node will never send feedback msg to another
node resulting in accumulation of dead tuples on another node. Is that
a possibility?
thanks
Shveta
On Wednesday, September 11, 2024 1:03 PM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Sep 11, 2024 at 10:15 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Wednesday, September 11, 2024 12:18 PM shveta malik
<shveta.malik@gmail.com> wrote:
~~
Another query is about 3 node setup. I couldn't figure out what
would be feedback_slots setting when it is not bidirectional, as in
consider the case where there are three nodes A,B,C. Node C is
subscribing to both Node A and Node B. Node A and Node B are the
ones doing concurrent "update" and "delete" which will both be
replicated to Node C. In this case what will be the feedback_slots
setting on Node C? We don't have any slots here which will be
replicating changes from Node C to Node A and Node C to Node B. This
is given in [3] in your first email ([1])Thanks for pointing this, the link was a bit misleading. I think the
solution proposed in this thread is only used to allow detecting
update_deleted reliably in a bidirectional cluster. For non-
bidirectional cases, it would be more tricky to predict the timing till whenshould we retain the dead tuples.
So in brief, this solution is only for bidrectional setup? For non-bidirectional,
feedback_slots is non-configurable and thus irrelevant.
Right.
Irrespective of above, if user ends up setting feedback_slot to some random but
existing slot which is not at all consuming changes, then it may so happen that
the node will never send feedback msg to another node resulting in
accumulation of dead tuples on another node. Is that a possibility?
Yes, It's possible. I think this is a common situation for this kind of user
specified options. Like the user DML will be blocked, if any inactive standby
names are added synchronous_standby_names.
Best Regards,
Hou zj
On Wed, Sep 11, 2024 at 8:32 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 10, 2024 7:25 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
One minor comment on 0003 ======================= 1. get_slot_confirmed_flush() { ... + /* + * To prevent concurrent slot dropping and creation while filtering the + * slots, take the ReplicationSlotControlLock outside of the loop. + */ + LWLockAcquire(ReplicationSlotControlLock, LW_SHARED); + + foreach_ptr(String, name, MySubscription->feedback_slots) { XLogRecPtr + confirmed_flush; ReplicationSlot *slot; + + slot = ValidateAndGetFeedbackSlot(strVal(name));Why do we need to validate slots each time here? Isn't it better to do it once?
I think it's possible that the slot was correct but changed or dropped later,
so it could be useful to give a warning in this case to hint user to adjust the
slots, otherwise, the xmin of the publisher's slot won't be advanced and might
cause dead tuples accumulation. This is similar to the checks we performed for
the slots in "synchronized_standby_slots". (E.g. StandbySlotsHaveCaughtup)
In the case of "synchronized_standby_slots", we seem to be invoking
such checks via StandbySlotsHaveCaughtup() when we need to wait for
WAL and also we have some optimizations that avoid the frequent
checking for validation checks. OTOH, this patch doesn't have any such
optimizations. We can optimize it by maintaining a local copy of
feedback slots to avoid looping all the slots each time (if this is
required, we can make it a top-up patch so that it can be reviewed
separately). I have also thought of maintaining the updated value of
confirmed_flush_lsn for feedback slots corresponding to a subscription
in shared memory but that seems tricky because then we have to
maintain slot->subscription mapping. Can you think of any other ways?
Having said that it is better to profile this in various scenarios
like by increasing the frequency of keep_alieve message and or in idle
subscriber cases where we try to send this new feedback message.
--
With Regards,
Amit Kapila.
On Wed, Sep 11, 2024 at 11:07 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wednesday, September 11, 2024 1:03 PM shveta malik <shveta.malik@gmail.com> wrote:
Another query is about 3 node setup. I couldn't figure out what
would be feedback_slots setting when it is not bidirectional, as in
consider the case where there are three nodes A,B,C. Node C is
subscribing to both Node A and Node B. Node A and Node B are the
ones doing concurrent "update" and "delete" which will both be
replicated to Node C. In this case what will be the feedback_slots
setting on Node C? We don't have any slots here which will be
replicating changes from Node C to Node A and Node C to Node B. This
is given in [3] in your first email ([1])Thanks for pointing this, the link was a bit misleading. I think the
solution proposed in this thread is only used to allow detecting
update_deleted reliably in a bidirectional cluster. For non-
bidirectional cases, it would be more tricky to predict the timing till whenshould we retain the dead tuples.
So in brief, this solution is only for bidrectional setup? For non-bidirectional,
feedback_slots is non-configurable and thus irrelevant.Right.
One possible idea to address the non-bidirectional case raised by
Shveta is to use a time-based cut-off to remove dead tuples. As
mentioned earlier in my email [1]/messages/by-id/CAA4eK1Lj-PWrP789KnKxZydisHajd38rSihWXO8MVBLDwxG1Kg@mail.gmail.com, we can define a new GUC parameter
say vacuum_committs_age which would indicate that we will allow rows
to be removed only if the modified time of the tuple as indicated by
committs module is greater than the vacuum_committs_age. We could keep
this parameter a table-level option without introducing a GUC as this
may not apply to all tables. I checked and found that some other
replication solutions like GoldenGate also allowed similar parameters
(tombstone_deletes) to be specified at table level [2]BEGIN DBMS_GOLDENGATE_ADM.ALTER_AUTO_CDR( schema_name => 'hr', table_name => 'employees', tombstone_deletes => TRUE); END; /. The other
advantage of allowing it at table level is that it won't hamper the
performance of hot-pruning or vacuum in general. Note, I am careful
here because to decide whether to remove a dead tuple or not we need
to compare its committs_time both during hot-pruning and vacuum.
Note that tombstones_deletes is a general concept used by replication
solutions to detect updated_deleted conflict and time-based purging is
recommended. See [3]https://en.wikipedia.org/wiki/Tombstone_(data_store)[4]https://docs.oracle.com/en/middleware/goldengate/core/19.1/oracle-db/automatic-conflict-detection-and-resolution1.html#GUID-423C6EE8-1C62-4085-899C-8454B8FB9C92. We previously discussed having tombstone
tables to keep the deleted records information but it was suggested to
prevent the vacuum from removing the required dead tuples as that
would be simpler than inventing a new kind of tables/store for
tombstone_deletes [5]/messages/by-id/e4cdb849-d647-4acf-aabe-7049ae170fbf@enterprisedb.com. So, we came up with the idea of feedback slots
discussed in this email but that didn't work out in all cases and
appears difficult to configure as pointed out by Shveta. So, now, we
are back to one of the other ideas [1]/messages/by-id/CAA4eK1Lj-PWrP789KnKxZydisHajd38rSihWXO8MVBLDwxG1Kg@mail.gmail.com discussed previously to solve
this problem.
Thoughts?
[1]: /messages/by-id/CAA4eK1Lj-PWrP789KnKxZydisHajd38rSihWXO8MVBLDwxG1Kg@mail.gmail.com
[2]: BEGIN DBMS_GOLDENGATE_ADM.ALTER_AUTO_CDR( schema_name => 'hr', table_name => 'employees', tombstone_deletes => TRUE); END; /
BEGIN
DBMS_GOLDENGATE_ADM.ALTER_AUTO_CDR(
schema_name => 'hr',
table_name => 'employees',
tombstone_deletes => TRUE);
END;
/
[3]: https://en.wikipedia.org/wiki/Tombstone_(data_store)
[4]: https://docs.oracle.com/en/middleware/goldengate/core/19.1/oracle-db/automatic-conflict-detection-and-resolution1.html#GUID-423C6EE8-1C62-4085-899C-8454B8FB9C92
[5]: /messages/by-id/e4cdb849-d647-4acf-aabe-7049ae170fbf@enterprisedb.com
--
With Regards,
Amit Kapila.
On Fri, Sep 13, 2024 at 11:38 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
So in brief, this solution is only for bidrectional setup? For non-bidirectional,
feedback_slots is non-configurable and thus irrelevant.Right.
One possible idea to address the non-bidirectional case raised by
Shveta is to use a time-based cut-off to remove dead tuples. As
mentioned earlier in my email [1], we can define a new GUC parameter
say vacuum_committs_age which would indicate that we will allow rows
to be removed only if the modified time of the tuple as indicated by
committs module is greater than the vacuum_committs_age. We could keep
this parameter a table-level option without introducing a GUC as this
may not apply to all tables. I checked and found that some other
replication solutions like GoldenGate also allowed similar parameters
(tombstone_deletes) to be specified at table level [2]. The other
advantage of allowing it at table level is that it won't hamper the
performance of hot-pruning or vacuum in general. Note, I am careful
here because to decide whether to remove a dead tuple or not we need
to compare its committs_time both during hot-pruning and vacuum.
+1 on the idea, but IIUC this value doesn’t need to be significant; it
can be limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other factors,
assuming clock skew has already been addressed.
This new parameter is necessary only for cases where an UPDATE and
DELETE on the same row occur concurrently, but the replication order
to a third node is not preserved, which could result in data
divergence. Consider the following example:
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); (10.01 AM)
T2: DELETE FROM t WHERE id = 1; (10.03 AM)
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; (10.02 AM)
Assume a third node (Node C) subscribes to both Node A and Node B. The
"correct" order of messages received by Node C would be T1-T3-T2, but
it could also receive them in the order T1-T2-T3, wherein sayT3 is
received with a lag of say 2 mins. In such a scenario, T3 should be
able to recognize that the row was deleted by T2 on Node C, thereby
detecting the update-deleted conflict and skipping the apply.
The 'vacuum_committs_age' parameter should account for this lag, which
could lead to the order reversal of UPDATE and DELETE operations.
Any subsequent attempt to update the same row after conflict detection
and resolution should not pose an issue. For example, if Node A
triggers the following at 10:20 AM:
UPDATE t SET value = 3 WHERE id = 1;
Since the row has already been deleted, the UPDATE will not proceed
and therefore will not generate a replication operation on the other
nodes, indicating that vacuum need not to preserve the dead row to
this far.
thanks
Shveta
On Fri, Sep 13, 2024 at 12:56 AM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, Sep 13, 2024 at 11:38 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
So in brief, this solution is only for bidrectional setup? For non-bidirectional,
feedback_slots is non-configurable and thus irrelevant.Right.
One possible idea to address the non-bidirectional case raised by
Shveta is to use a time-based cut-off to remove dead tuples. As
mentioned earlier in my email [1], we can define a new GUC parameter
say vacuum_committs_age which would indicate that we will allow rows
to be removed only if the modified time of the tuple as indicated by
committs module is greater than the vacuum_committs_age. We could keep
this parameter a table-level option without introducing a GUC as this
may not apply to all tables. I checked and found that some other
replication solutions like GoldenGate also allowed similar parameters
(tombstone_deletes) to be specified at table level [2]. The other
advantage of allowing it at table level is that it won't hamper the
performance of hot-pruning or vacuum in general. Note, I am careful
here because to decide whether to remove a dead tuple or not we need
to compare its committs_time both during hot-pruning and vacuum.+1 on the idea,
I agree that this idea is much simpler than the idea originally
proposed in this thread.
IIUC vacuum_committs_age specifies a time rather than an XID age. But
how can we implement it? If it ends up affecting the vacuum cutoff, we
should be careful not to end up with the same result of
vacuum_defer_cleanup_age that was discussed before[1]/messages/by-id/20230317230930.nhsgk3qfk7f4axls@awork3.anarazel.de. Also, I think
the implementation needs not to affect the performance of
ComputeXidHorizons().
but IIUC this value doesn’t need to be significant; it
can be limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other factors,
assuming clock skew has already been addressed.
I think that in a non-bidirectional case the value could need to be a
large number. Is that right?
Regards,
[1]: /messages/by-id/20230317230930.nhsgk3qfk7f4axls@awork3.anarazel.de
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tue, Sep 17, 2024 at 6:08 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Sep 13, 2024 at 12:56 AM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, Sep 13, 2024 at 11:38 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
So in brief, this solution is only for bidrectional setup? For non-bidirectional,
feedback_slots is non-configurable and thus irrelevant.Right.
One possible idea to address the non-bidirectional case raised by
Shveta is to use a time-based cut-off to remove dead tuples. As
mentioned earlier in my email [1], we can define a new GUC parameter
say vacuum_committs_age which would indicate that we will allow rows
to be removed only if the modified time of the tuple as indicated by
committs module is greater than the vacuum_committs_age. We could keep
this parameter a table-level option without introducing a GUC as this
may not apply to all tables. I checked and found that some other
replication solutions like GoldenGate also allowed similar parameters
(tombstone_deletes) to be specified at table level [2]. The other
advantage of allowing it at table level is that it won't hamper the
performance of hot-pruning or vacuum in general. Note, I am careful
here because to decide whether to remove a dead tuple or not we need
to compare its committs_time both during hot-pruning and vacuum.+1 on the idea,
I agree that this idea is much simpler than the idea originally
proposed in this thread.IIUC vacuum_committs_age specifies a time rather than an XID age.
Your understanding is correct that vacuum_committs_age specifies a time.
But
how can we implement it? If it ends up affecting the vacuum cutoff, we
should be careful not to end up with the same result of
vacuum_defer_cleanup_age that was discussed before[1]. Also, I think
the implementation needs not to affect the performance of
ComputeXidHorizons().
I haven't thought about the implementation details yet but I think
during pruning (for example in heap_prune_satisfies_vacuum()), apart
from checking if the tuple satisfies
HeapTupleSatisfiesVacuumHorizon(), we should also check if the tuple's
committs is greater than configured vacuum_committs_age (for the
table) to decide whether tuple can be removed. One thing to consider
is what to do in case of aggressive vacuum where we expect
relfrozenxid to be advanced to FreezeLimit (at a minimum). We may want
to just ignore vacuum_committs_age during aggressive vacuum and LOG if
we end up removing some tuple. This will allow users to retain deleted
tuples by respecting the freeze limits which also avoid xid_wrap
around. I think we can't retain tuples forever if the user
misconfigured vacuum_committs_age and to avoid that we can keep the
maximum limit on this parameter to say an hour or so. Also, users can
tune freeze parameters if they want to retain tuples for longer.
but IIUC this value doesn’t need to be significant; it
can be limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other factors,
assuming clock skew has already been addressed.I think that in a non-bidirectional case the value could need to be a
large number. Is that right?
As per my understanding, even for non-bidirectional cases, the value
should be small. For example, in the case, pointed out by Shveta [1]/messages/by-id/CAJpy0uAzzOzhXGH-zBc7Zt8ndXRf6r4OnLzgRrHyf8cvd+fpwg@mail.gmail.com,
where the updates from 2 nodes are received by a third node, this
setting is expected to be small. This setting primarily deals with
concurrent transactions on multiple nodes, so it should be small but I
could be missing something.
[1]: /messages/by-id/CAJpy0uAzzOzhXGH-zBc7Zt8ndXRf6r4OnLzgRrHyf8cvd+fpwg@mail.gmail.com
--
With Regards,
Amit Kapila.
On Mon, Sep 16, 2024 at 11:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 17, 2024 at 6:08 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Sep 13, 2024 at 12:56 AM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, Sep 13, 2024 at 11:38 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
So in brief, this solution is only for bidrectional setup? For non-bidirectional,
feedback_slots is non-configurable and thus irrelevant.Right.
One possible idea to address the non-bidirectional case raised by
Shveta is to use a time-based cut-off to remove dead tuples. As
mentioned earlier in my email [1], we can define a new GUC parameter
say vacuum_committs_age which would indicate that we will allow rows
to be removed only if the modified time of the tuple as indicated by
committs module is greater than the vacuum_committs_age. We could keep
this parameter a table-level option without introducing a GUC as this
may not apply to all tables. I checked and found that some other
replication solutions like GoldenGate also allowed similar parameters
(tombstone_deletes) to be specified at table level [2]. The other
advantage of allowing it at table level is that it won't hamper the
performance of hot-pruning or vacuum in general. Note, I am careful
here because to decide whether to remove a dead tuple or not we need
to compare its committs_time both during hot-pruning and vacuum.+1 on the idea,
I agree that this idea is much simpler than the idea originally
proposed in this thread.IIUC vacuum_committs_age specifies a time rather than an XID age.
Your understanding is correct that vacuum_committs_age specifies a time.
But
how can we implement it? If it ends up affecting the vacuum cutoff, we
should be careful not to end up with the same result of
vacuum_defer_cleanup_age that was discussed before[1]. Also, I think
the implementation needs not to affect the performance of
ComputeXidHorizons().I haven't thought about the implementation details yet but I think
during pruning (for example in heap_prune_satisfies_vacuum()), apart
from checking if the tuple satisfies
HeapTupleSatisfiesVacuumHorizon(), we should also check if the tuple's
committs is greater than configured vacuum_committs_age (for the
table) to decide whether tuple can be removed.
Sounds very costly. I think we need to do performance tests. Even if
the vacuum gets slower only on the particular table having the
vacuum_committs_age setting, it would affect overall autovacuum
performance. Also, it would affect HOT pruning performance.
but IIUC this value doesn’t need to be significant; it
can be limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other factors,
assuming clock skew has already been addressed.I think that in a non-bidirectional case the value could need to be a
large number. Is that right?As per my understanding, even for non-bidirectional cases, the value
should be small. For example, in the case, pointed out by Shveta [1],
where the updates from 2 nodes are received by a third node, this
setting is expected to be small. This setting primarily deals with
concurrent transactions on multiple nodes, so it should be small but I
could be missing something.
I might be missing something but the scenario I was thinking of is
something below.
Suppose that we setup uni-directional logical replication between Node
A and Node B (e.g., Node A -> Node B) and both nodes have the same row
with key = 1:
Node A:
T1: UPDATE t SET val = 2 WHERE key = 1; (10:00 AM)
-> This change is applied on Node B at 10:01 AM.
Node B:
T2: DELETE FROM t WHERE key = 1; (05:00 AM)
If a vacuum runs on Node B at 06:00 AM, the change of T1 coming from
Node A would raise an "update_missing" conflict. On the other hand, if
a vacuum runs on Node B at 11:00 AM, the change would raise an
"update_deleted" conflict. It looks whether we detect an
"update_deleted" or an "updated_missing" depends on the timing of
vacuum, and to avoid such a situation, we would need to set
vacuum_committs_age to more than 5 hours.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tue, Sep 17, 2024 at 11:24 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Sep 16, 2024 at 11:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 17, 2024 at 6:08 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I haven't thought about the implementation details yet but I think
during pruning (for example in heap_prune_satisfies_vacuum()), apart
from checking if the tuple satisfies
HeapTupleSatisfiesVacuumHorizon(), we should also check if the tuple's
committs is greater than configured vacuum_committs_age (for the
table) to decide whether tuple can be removed.Sounds very costly. I think we need to do performance tests. Even if
the vacuum gets slower only on the particular table having the
vacuum_committs_age setting, it would affect overall autovacuum
performance. Also, it would affect HOT pruning performance.
Agreed that we should do some performance testing and additionally
think of any better way to implement. I think the cost won't be much
if the tuples to be removed are from a single transaction because the
required commit_ts information would be cached but when the tuples are
from different transactions, we could see a noticeable impact. We need
to test to say anything concrete on this.
but IIUC this value doesn’t need to be significant; it
can be limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other factors,
assuming clock skew has already been addressed.I think that in a non-bidirectional case the value could need to be a
large number. Is that right?As per my understanding, even for non-bidirectional cases, the value
should be small. For example, in the case, pointed out by Shveta [1],
where the updates from 2 nodes are received by a third node, this
setting is expected to be small. This setting primarily deals with
concurrent transactions on multiple nodes, so it should be small but I
could be missing something.I might be missing something but the scenario I was thinking of is
something below.Suppose that we setup uni-directional logical replication between Node
A and Node B (e.g., Node A -> Node B) and both nodes have the same row
with key = 1:Node A:
T1: UPDATE t SET val = 2 WHERE key = 1; (10:00 AM)
-> This change is applied on Node B at 10:01 AM.Node B:
T2: DELETE FROM t WHERE key = 1; (05:00 AM)If a vacuum runs on Node B at 06:00 AM, the change of T1 coming from
Node A would raise an "update_missing" conflict. On the other hand, if
a vacuum runs on Node B at 11:00 AM, the change would raise an
"update_deleted" conflict. It looks whether we detect an
"update_deleted" or an "updated_missing" depends on the timing of
vacuum, and to avoid such a situation, we would need to set
vacuum_committs_age to more than 5 hours.
Yeah, in this case, it would detect a different conflict (if we don't
set vacuum_committs_age to greater than 5 hours) but as per my
understanding, the primary purpose of conflict detection and
resolution is to avoid data inconsistency in a bi-directional setup.
Assume, in the above case it is a bi-directional setup, then we want
to have the same data in both nodes. Now, if there are other cases
like the one you mentioned that require to detect the conflict
reliably than I agree this value could be large and probably not the
best way to achieve it. I think we can mention in the docs that the
primary purpose of this is to achieve data consistency among
bi-directional kind of setups.
Having said that even in the above case, the result should be the same
whether the vacuum has removed the row or not. Say, if the vacuum has
not yet removed the row (due to vacuum_committs_age or otherwise) then
also because the incoming update has a later timestamp, we will
convert the update to insert as per last_update_wins resolution
method, so the conflict will be considered as update_missing. And,
say, the vacuum has removed the row and the conflict detected is
update_missing, then also we will convert the update to insert. In
short, if UPDATE has lower commit-ts, DELETE should win and if UPDATE
has higher commit-ts, UPDATE should win.
So, we can expect data consistency in bidirectional cases and expect a
deterministic behavior in other cases (e.g. the final data in a table
does not depend on the order of applying the transactions from other
nodes).
--
With Regards,
Amit Kapila.
On Tue, Sep 17, 2024 at 9:29 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 17, 2024 at 11:24 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Sep 16, 2024 at 11:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 17, 2024 at 6:08 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I haven't thought about the implementation details yet but I think
during pruning (for example in heap_prune_satisfies_vacuum()), apart
from checking if the tuple satisfies
HeapTupleSatisfiesVacuumHorizon(), we should also check if the tuple's
committs is greater than configured vacuum_committs_age (for the
table) to decide whether tuple can be removed.Sounds very costly. I think we need to do performance tests. Even if
the vacuum gets slower only on the particular table having the
vacuum_committs_age setting, it would affect overall autovacuum
performance. Also, it would affect HOT pruning performance.Agreed that we should do some performance testing and additionally
think of any better way to implement. I think the cost won't be much
if the tuples to be removed are from a single transaction because the
required commit_ts information would be cached but when the tuples are
from different transactions, we could see a noticeable impact. We need
to test to say anything concrete on this.
Agreed.
but IIUC this value doesn’t need to be significant; it
can be limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other factors,
assuming clock skew has already been addressed.I think that in a non-bidirectional case the value could need to be a
large number. Is that right?As per my understanding, even for non-bidirectional cases, the value
should be small. For example, in the case, pointed out by Shveta [1],
where the updates from 2 nodes are received by a third node, this
setting is expected to be small. This setting primarily deals with
concurrent transactions on multiple nodes, so it should be small but I
could be missing something.I might be missing something but the scenario I was thinking of is
something below.Suppose that we setup uni-directional logical replication between Node
A and Node B (e.g., Node A -> Node B) and both nodes have the same row
with key = 1:Node A:
T1: UPDATE t SET val = 2 WHERE key = 1; (10:00 AM)
-> This change is applied on Node B at 10:01 AM.Node B:
T2: DELETE FROM t WHERE key = 1; (05:00 AM)If a vacuum runs on Node B at 06:00 AM, the change of T1 coming from
Node A would raise an "update_missing" conflict. On the other hand, if
a vacuum runs on Node B at 11:00 AM, the change would raise an
"update_deleted" conflict. It looks whether we detect an
"update_deleted" or an "updated_missing" depends on the timing of
vacuum, and to avoid such a situation, we would need to set
vacuum_committs_age to more than 5 hours.Yeah, in this case, it would detect a different conflict (if we don't
set vacuum_committs_age to greater than 5 hours) but as per my
understanding, the primary purpose of conflict detection and
resolution is to avoid data inconsistency in a bi-directional setup.
Assume, in the above case it is a bi-directional setup, then we want
to have the same data in both nodes. Now, if there are other cases
like the one you mentioned that require to detect the conflict
reliably than I agree this value could be large and probably not the
best way to achieve it. I think we can mention in the docs that the
primary purpose of this is to achieve data consistency among
bi-directional kind of setups.Having said that even in the above case, the result should be the same
whether the vacuum has removed the row or not. Say, if the vacuum has
not yet removed the row (due to vacuum_committs_age or otherwise) then
also because the incoming update has a later timestamp, we will
convert the update to insert as per last_update_wins resolution
method, so the conflict will be considered as update_missing. And,
say, the vacuum has removed the row and the conflict detected is
update_missing, then also we will convert the update to insert. In
short, if UPDATE has lower commit-ts, DELETE should win and if UPDATE
has higher commit-ts, UPDATE should win.So, we can expect data consistency in bidirectional cases and expect a
deterministic behavior in other cases (e.g. the final data in a table
does not depend on the order of applying the transactions from other
nodes).
Agreed.
I think that such a time-based configuration parameter would be a
reasonable solution. The current concerns are that it might affect
vacuum performance and lead to a similar bug we had with
vacuum_defer_cleanup_age.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
-----Original Message-----
From: Masahiko Sawada <sawada.mshk@gmail.com>
Sent: Friday, September 20, 2024 2:49 AM
To: Amit Kapila <amit.kapila16@gmail.com>
Cc: shveta malik <shveta.malik@gmail.com>; Hou, Zhijie/侯 志杰
<houzj.fnst@fujitsu.com>; pgsql-hackers <pgsql-hackers@postgresql.org>
Subject: Re: Conflict detection for update_deleted in logical replicationOn Tue, Sep 17, 2024 at 9:29 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Tue, Sep 17, 2024 at 11:24 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Mon, Sep 16, 2024 at 11:53 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:
On Tue, Sep 17, 2024 at 6:08 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I haven't thought about the implementation details yet but I think
during pruning (for example in heap_prune_satisfies_vacuum()),
apart from checking if the tuple satisfies
HeapTupleSatisfiesVacuumHorizon(), we should also check if the
tuple's committs is greater than configured vacuum_committs_age
(for the
table) to decide whether tuple can be removed.Sounds very costly. I think we need to do performance tests. Even if
the vacuum gets slower only on the particular table having the
vacuum_committs_age setting, it would affect overall autovacuum
performance. Also, it would affect HOT pruning performance.Agreed that we should do some performance testing and additionally
think of any better way to implement. I think the cost won't be much
if the tuples to be removed are from a single transaction because the
required commit_ts information would be cached but when the tuples are
from different transactions, we could see a noticeable impact. We need
to test to say anything concrete on this.Agreed.
but IIUC this value doesn’t need to be significant; it can be
limited to just a few minutes. The one which is sufficient to
handle replication delays caused by network lag or other
factors, assuming clock skew has already been addressed.I think that in a non-bidirectional case the value could need to
be a large number. Is that right?As per my understanding, even for non-bidirectional cases, the
value should be small. For example, in the case, pointed out by
Shveta [1], where the updates from 2 nodes are received by a third
node, this setting is expected to be small. This setting primarily
deals with concurrent transactions on multiple nodes, so it should
be small but I could be missing something.I might be missing something but the scenario I was thinking of is
something below.Suppose that we setup uni-directional logical replication between
Node A and Node B (e.g., Node A -> Node B) and both nodes have the
same row with key = 1:Node A:
T1: UPDATE t SET val = 2 WHERE key = 1; (10:00 AM)
-> This change is applied on Node B at 10:01 AM.Node B:
T2: DELETE FROM t WHERE key = 1; (05:00 AM)If a vacuum runs on Node B at 06:00 AM, the change of T1 coming from
Node A would raise an "update_missing" conflict. On the other hand,
if a vacuum runs on Node B at 11:00 AM, the change would raise an
"update_deleted" conflict. It looks whether we detect an
"update_deleted" or an "updated_missing" depends on the timing of
vacuum, and to avoid such a situation, we would need to set
vacuum_committs_age to more than 5 hours.Yeah, in this case, it would detect a different conflict (if we don't
set vacuum_committs_age to greater than 5 hours) but as per my
understanding, the primary purpose of conflict detection and
resolution is to avoid data inconsistency in a bi-directional setup.
Assume, in the above case it is a bi-directional setup, then we want
to have the same data in both nodes. Now, if there are other cases
like the one you mentioned that require to detect the conflict
reliably than I agree this value could be large and probably not the
best way to achieve it. I think we can mention in the docs that the
primary purpose of this is to achieve data consistency among
bi-directional kind of setups.Having said that even in the above case, the result should be the same
whether the vacuum has removed the row or not. Say, if the vacuum has
not yet removed the row (due to vacuum_committs_age or otherwise) then
also because the incoming update has a later timestamp, we will
convert the update to insert as per last_update_wins resolution
method, so the conflict will be considered as update_missing. And,
say, the vacuum has removed the row and the conflict detected is
update_missing, then also we will convert the update to insert. In
short, if UPDATE has lower commit-ts, DELETE should win and if UPDATE
has higher commit-ts, UPDATE should win.So, we can expect data consistency in bidirectional cases and expect a
deterministic behavior in other cases (e.g. the final data in a table
does not depend on the order of applying the transactions from other
nodes).Agreed.
I think that such a time-based configuration parameter would be a reasonable
solution. The current concerns are that it might affect vacuum performance and
lead to a similar bug we had with vacuum_defer_cleanup_age.
Thanks for the feedback!
I am working on the POC patch and doing some initial performance tests on this idea.
I will share the results after finishing.
Apart from the vacuum_defer_cleanup_age idea. we’ve given more thought to our
approach for retaining dead tuples and have come up with another idea that can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce the performance
impact. Thanks a lot to Amit for off-list discussion.
The concept of the new idea is that, the dead tuples are only useful to detect
conflicts when applying *concurrent* transactions from remotes. Any subsequent
UPDATE from a remote node after removing the dead tuples should have a later
timestamp, meaning it's reasonable to detect an update_missing scenario and
convert the UPDATE to an INSERT when applying it.
To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin after confirming
that all the concurrent transaction on publisher has been applied locally.
The process of advancing the slot.xmin could be:
1) the apply worker call GetRunningTransactionData() to get the
'oldestRunningXid' and consider this as 'candidate_xmin'.
2) the apply worker send a new message to walsender to request the latest wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedback message or
extend the existing keepalive message(e,g extends the requestReply bit in
keepalive message to add a 'request_wal_position' value)
3) The apply worker can continue to apply changes. After applying all the WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It can be effective for both bidirectional and
non-bidirectional replication cases.
We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending to detect
update-delete conflicts should set retain_dead_tuples to true.
The following explains how it works in different cases to achieve data
consistency:
--
2 nodes, bidirectional case 1:
--
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AM
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AM
subscription retain_dead_tuples = true/false
After executing T2, the apply worker on Node A will check the latest wal flush
location on Node B. Till that time, the T3 should have finished, so the xmin
will be advanced only after applying the WALs that is later than T3. So, the
dead tuple will not be removed before applying the T3, which means the
update_delete can be detected.
--
2 nodes, bidirectional case 2:
--
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.01 AM
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.02 AM
After executing T2, the apply worker on Node A will request the latest wal
flush location on Node B. And the T3 is either running concurrently or has not
started. In both cases, the T3 must have a later timestamp. So, even if the
dead tuple is removed in this cases and update_missing is detected, the default
resolution is to convert UDPATE to INSERT which is OK because the data are
still consistent on Node A and B.
--
3 nodes, non-bidirectional, Node C subscribes to both Node A and Node B:
--
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.01 AM
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.02 AM
Node C:
apply T1, T2, T3
After applying T2, the apply worker on Node C will check the latest wal flush
location on Node B. Till that time, the T3 should have finished, so the xmin
will be advanced only after applying the WALs that is later than T3. So, the
dead tuple will not be removed before applying the T3, which means the
update_delete can be detected.
Your feedback on this idea would be greatly appreciated.
Best Regards,
Hou zj
On Friday, September 20, 2024 10:55 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Friday, September 20, 2024 2:49 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I think that such a time-based configuration parameter would be a
reasonable solution. The current concerns are that it might affect
vacuum performance and lead to a similar bug we had withvacuum_defer_cleanup_age.
Thanks for the feedback!
I am working on the POC patch and doing some initial performance tests on
this idea.
I will share the results after finishing.Apart from the vacuum_defer_cleanup_age idea. we’ve given more thought to
our approach for retaining dead tuples and have come up with another idea that
can reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce the performance
impact. Thanks a lot to Amit for off-list discussion.The concept of the new idea is that, the dead tuples are only useful to detect
conflicts when applying *concurrent* transactions from remotes. Any
subsequent UPDATE from a remote node after removing the dead tuples
should have a later timestamp, meaning it's reasonable to detect an
update_missing scenario and convert the UPDATE to an INSERT when
applying it.To achieve above, we can create an additional replication slot on the subscriber
side, maintained by the apply worker. This slot is used to retain the dead tuples.
The apply worker will advance the slot.xmin after confirming that all the
concurrent transaction on publisher has been applied locally.The process of advancing the slot.xmin could be:
1) the apply worker call GetRunningTransactionData() to get the
'oldestRunningXid' and consider this as 'candidate_xmin'.
2) the apply worker send a new message to walsender to request the latest wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedback
message or extend the existing keepalive message(e,g extends the
requestReply bit in keepalive message to add a 'request_wal_position' value)
3) The apply worker can continue to apply changes. After applying all the WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It can be effective for both bidirectional and
non-bidirectional replication cases.We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending to detect
update-delete conflicts should set retain_dead_tuples to true.The following explains how it works in different cases to achieve data
consistency:
...
--
3 nodes, non-bidirectional, Node C subscribes to both Node A and Node B:
--
Sorry for a typo here, the time of T2 and T3 were reversed.
Please see the following correction:
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.01 AM
Here T2 should be at ts=10.02 AM
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.02 AM
T3 should be at ts=10.01 AM
Node C:
apply T1, T2, T3After applying T2, the apply worker on Node C will check the latest wal flush
location on Node B. Till that time, the T3 should have finished, so the xmin will
be advanced only after applying the WALs that is later than T3. So, the dead
tuple will not be removed before applying the T3, which means the
update_delete can be detected.Your feedback on this idea would be greatly appreciated.
Best Regards,
Hou zj
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another idea that can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce the performance
impact. Thanks a lot to Amit for off-list discussion.The concept of the new idea is that, the dead tuples are only useful to detect
conflicts when applying *concurrent* transactions from remotes. Any subsequent
UPDATE from a remote node after removing the dead tuples should have a later
timestamp, meaning it's reasonable to detect an update_missing scenario and
convert the UPDATE to an INSERT when applying it.To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin after confirming
that all the concurrent transaction on publisher has been applied locally.The process of advancing the slot.xmin could be:
1) the apply worker call GetRunningTransactionData() to get the
'oldestRunningXid' and consider this as 'candidate_xmin'.
2) the apply worker send a new message to walsender to request the latest wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedback message or
extend the existing keepalive message(e,g extends the requestReply bit in
keepalive message to add a 'request_wal_position' value)
3) The apply worker can continue to apply changes. After applying all the WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It can be effective for both bidirectional and
non-bidirectional replication cases.We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending to detect
update-delete conflicts should set retain_dead_tuples to true.
As each apply worker needs a separate slot to retain deleted rows, the
requirement for slots will increase. The other possibility is to
maintain one slot by launcher or some other central process that
traverses all subscriptions, remember the ones marked with
retain_dead_rows (let's call this list as retain_sub_list). Then using
running_transactions get the oldest running_xact, and then get the
remote flush location from the other node (publisher node) and store
those as candidate values (candidate_xmin and
candidate_remote_wal_lsn) in slot. We can probably reuse existing
candidate variables of the slot. Next, we can check the remote_flush
locations from all the origins corresponding in retain_sub_list and if
all are ahead of candidate_remote_wal_lsn, we can update the slot's
xmin to candidate_xmin.
I think in the above idea we can an optimization to combine the
request for remote wal LSN from different subscriptions pointing to
the same node to avoid sending multiple requests to the same node. I
am not sure if using pg_subscription.subconninfo is sufficient for
this, if not we can probably leave this optimization.
If this idea is feasible then it would reduce the number of slots
required to retain the deleted rows but the launcher needs to get the
remote wal location corresponding to each publisher node. There are
two ways to achieve that (a) launcher requests one of the apply
workers corresponding to subscriptions pointing to the same publisher
node to get this information; (b) launcher launches another worker to
get the remote wal flush location.
--
With Regards,
Amit Kapila.
Hi,
Thank you for considering another idea.
On Fri, Sep 20, 2024 at 2:46 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another idea that can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce the performance
impact. Thanks a lot to Amit for off-list discussion.The concept of the new idea is that, the dead tuples are only useful to detect
conflicts when applying *concurrent* transactions from remotes. Any subsequent
UPDATE from a remote node after removing the dead tuples should have a later
timestamp, meaning it's reasonable to detect an update_missing scenario and
convert the UPDATE to an INSERT when applying it.To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin after confirming
that all the concurrent transaction on publisher has been applied locally.
The replication slot used for this purpose will be a physical one or
logical one? And IIUC such a slot doesn't need to retain WAL but if we
do that, how do we advance the LSN of the slot?
2) the apply worker send a new message to walsender to request the latest wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedback message or
extend the existing keepalive message(e,g extends the requestReply bit in
keepalive message to add a 'request_wal_position' value)
The apply worker sends a keepalive message when it didn't receive
anything more than wal_receiver_timeout / 2. So in a very active
system, we cannot rely on piggybacking new information to the
keepalive messages to get the latest remote flush LSN.
3) The apply worker can continue to apply changes. After applying all the WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It can be effective for both bidirectional and
non-bidirectional replication cases.We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending to detect
update-delete conflicts should set retain_dead_tuples to true.
I'm still studying this idea but let me confirm the following scenario.
Suppose both Node-A and Node-B have the same row (1,1) in table t, and
XIDs and commit LSNs of T2 and T3 are the following:
Node A
T2: DELETE FROM t WHERE id = 1 (10:02 AM) XID:100, commit-LSN:1000
Node B
T3: UPDATE t SET value = 2 WHERE id 1 (10:01 AM) XID:500, commit-LSN:5000
Further suppose that it's now 10:05 AM, and the latest XID and the
latest flush WAL position of Node-A and Node-B are following:
Node A
current XID: 300
latest flush LSN; 3000
Node B
current XID: 700
latest flush LSN: 7000
Both T2 and T3 are NOT sent to Node B and Node A yet, respectively
(i.e., the logical replication is delaying for 5 min).
Consider the following scenario:
1. The apply worker on Node-A calls GetRunningTransactionData() and
gets 301 (set as candidate_xmin).
2. The apply worker on Node-A requests the latest WAL flush position
from Node-B, and gets 7000 (set as candidate_remote_wal_lsn).
3. T2 is applied on Node-B, and the latest flush position of Node-B is now 8000.
4. The apply worker on Node-A continues applying changes, and applies
the transactions up to remote (commit) LSN 7100.
5. Now that the apply worker on Node-A applied all changes smaller
than candidate_remote_wal_lsn (7000), it increases the slot.xmin to
301 (candidate_xmin).
6. On Node-A, vacuum runs and physically removes the tuple that was
deleted by T2.
Here, on Node-B, there might be a transition between LSN 7100 and 8000
that might require the tuple that is deleted by T2.
For example, "UPDATE t SET value = 3 WHERE id = 1" (say T4) is
executed on Node-B at LSN 7200, and it's sent to Node-A after step 6.
On Node-A, whether we detect "update_deleted" or "update_missing"
still depends on when vacuum removes the tuple deleted by T2.
If applying T4 raises an "update_missing" (i.e. the changes are
applied in the order of T2->T3->(vacuum)->T4), it converts into an
insert, resulting in the table having a row with value = 3.
If applying T4 raises an "update_deleted" (i.e. the changes are
applied in the order of T2->T3->T4->(vacuum)), it's skipped, resulting
in the table having no row.
On the other hand, in this scenario, Node-B applies changes in the
order of T3->T4->T2, and applying T2 raises a "delete_origin_differ",
resulting in the table having a row with val=3 (assuming
latest_committs_win is the default resolver for this confliction).
Please confirm this scenario as I might be missing something.
As each apply worker needs a separate slot to retain deleted rows, the
requirement for slots will increase. The other possibility is to
maintain one slot by launcher or some other central process that
traverses all subscriptions, remember the ones marked with
retain_dead_rows (let's call this list as retain_sub_list). Then using
running_transactions get the oldest running_xact, and then get the
remote flush location from the other node (publisher node) and store
those as candidate values (candidate_xmin and
candidate_remote_wal_lsn) in slot. We can probably reuse existing
candidate variables of the slot. Next, we can check the remote_flush
locations from all the origins corresponding in retain_sub_list and if
all are ahead of candidate_remote_wal_lsn, we can update the slot's
xmin to candidate_xmin.
Does it mean that we use one candiate_remote_wal_lsn in a slot for all
subscriptions (in retain_sub_list)? IIUC candiate_remote_wal_lsn is a
LSN of one of publishers, so other publishers could have completely
different LSNs. How do we compare the candidate_remote_wal_lsn to
remote_flush locations from all the origins?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tuesday, September 24, 2024 5:05 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Thank you for considering another idea.
Thanks for reviewing the idea!
On Fri, Sep 20, 2024 at 2:46 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another ideathat can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce theperformance
impact. Thanks a lot to Amit for off-list discussion.
The concept of the new idea is that, the dead tuples are only useful to
detect
conflicts when applying *concurrent* transactions from remotes. Any
subsequent
UPDATE from a remote node after removing the dead tuples should have a
later
timestamp, meaning it's reasonable to detect an update_missing scenario
and
convert the UPDATE to an INSERT when applying it.
To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin afterconfirming
that all the concurrent transaction on publisher has been applied locally.
The replication slot used for this purpose will be a physical one or
logical one? And IIUC such a slot doesn't need to retain WAL but if we
do that, how do we advance the LSN of the slot?
I think it would be a logical slot. We can keep the
restart_lsn/confirmed_flush_lsn as invalid because we don't need to retain the
WALs for decoding purpose.
2) the apply worker send a new message to walsender to request the latest
wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedbackmessage or
extend the existing keepalive message(e,g extends the requestReply bit in
keepalive message to add a 'request_wal_position' value)The apply worker sends a keepalive message when it didn't receive
anything more than wal_receiver_timeout / 2. So in a very active
system, we cannot rely on piggybacking new information to the
keepalive messages to get the latest remote flush LSN.
Right. I think we need to send this new message at some interval independent of
wal_receiver_timeout.
3) The apply worker can continue to apply changes. After applying all the
WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all
concurrent
transactions have been applied. It can be effective for both bidirectional
and
non-bidirectional replication cases.
We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending todetect
update-delete conflicts should set retain_dead_tuples to true.
I'm still studying this idea but let me confirm the following scenario.
Suppose both Node-A and Node-B have the same row (1,1) in table t, and
XIDs and commit LSNs of T2 and T3 are the following:Node A
T2: DELETE FROM t WHERE id = 1 (10:02 AM) XID:100, commit-LSN:1000Node B
T3: UPDATE t SET value = 2 WHERE id 1 (10:01 AM) XID:500,
commit-LSN:5000Further suppose that it's now 10:05 AM, and the latest XID and the
latest flush WAL position of Node-A and Node-B are following:Node A
current XID: 300
latest flush LSN; 3000Node B
current XID: 700
latest flush LSN: 7000Both T2 and T3 are NOT sent to Node B and Node A yet, respectively
(i.e., the logical replication is delaying for 5 min).Consider the following scenario:
1. The apply worker on Node-A calls GetRunningTransactionData() and
gets 301 (set as candidate_xmin).
2. The apply worker on Node-A requests the latest WAL flush position
from Node-B, and gets 7000 (set as candidate_remote_wal_lsn).
3. T2 is applied on Node-B, and the latest flush position of Node-B is now 8000.
4. The apply worker on Node-A continues applying changes, and applies
the transactions up to remote (commit) LSN 7100.
5. Now that the apply worker on Node-A applied all changes smaller
than candidate_remote_wal_lsn (7000), it increases the slot.xmin to
301 (candidate_xmin).
6. On Node-A, vacuum runs and physically removes the tuple that was
deleted by T2.Here, on Node-B, there might be a transition between LSN 7100 and 8000
that might require the tuple that is deleted by T2.For example, "UPDATE t SET value = 3 WHERE id = 1" (say T4) is
executed on Node-B at LSN 7200, and it's sent to Node-A after step 6.
On Node-A, whether we detect "update_deleted" or "update_missing"
still depends on when vacuum removes the tuple deleted by T2.
I think in this case, no matter we detect "update_delete" or "update_missing",
the final data is the same. Because T4's commit timestamp should be later than
T2 on node A, so in the case of "update_deleted", it will compare the commit
timestamp of the deleted tuple's xmax with T4's timestamp, and T4 should win,
which means we will convert the update into insert and apply. Even if the
deleted tuple is deleted and "update_missing" is detected, the update will
still be converted into insert and applied. So, the result is the same.
If applying T4 raises an "update_missing" (i.e. the changes are
applied in the order of T2->T3->(vacuum)->T4), it converts into an
insert, resulting in the table having a row with value = 3.If applying T4 raises an "update_deleted" (i.e. the changes are
applied in the order of T2->T3->T4->(vacuum)), it's skipped, resulting
in the table having no row.On the other hand, in this scenario, Node-B applies changes in the
order of T3->T4->T2, and applying T2 raises a "delete_origin_differ",
resulting in the table having a row with val=3 (assuming
latest_committs_win is the default resolver for this confliction).Please confirm this scenario as I might be missing something.
As explained above, I think the data can be consistent in this case as well.
Best Regards,
Hou zj
On Tue, Sep 24, 2024 at 2:35 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
As each apply worker needs a separate slot to retain deleted rows, the
requirement for slots will increase. The other possibility is to
maintain one slot by launcher or some other central process that
traverses all subscriptions, remember the ones marked with
retain_dead_rows (let's call this list as retain_sub_list). Then using
running_transactions get the oldest running_xact, and then get the
remote flush location from the other node (publisher node) and store
those as candidate values (candidate_xmin and
candidate_remote_wal_lsn) in slot. We can probably reuse existing
candidate variables of the slot. Next, we can check the remote_flush
locations from all the origins corresponding in retain_sub_list and if
all are ahead of candidate_remote_wal_lsn, we can update the slot's
xmin to candidate_xmin.Does it mean that we use one candiate_remote_wal_lsn in a slot for all
subscriptions (in retain_sub_list)? IIUC candiate_remote_wal_lsn is a
LSN of one of publishers, so other publishers could have completely
different LSNs. How do we compare the candidate_remote_wal_lsn to
remote_flush locations from all the origins?
This should be an array/list with one element per publisher. We can
copy candidate_xmin to actual xmin only when the
candiate_remote_wal_lsn's corresponding to all publishers have been
applied aka their remote_flush locations (present in origins) are
ahead. The advantages I see with this are (a) reduces the number of
slots required to achieve the retention of deleted rows for conflict
detection, (b) in some cases we can avoid sending messages to the
publisher because with this we only need to send message to a
particular publisher once rather than by all the apply workers
corresponding to same publisher node.
--
With Regards,
Amit Kapila.
On Tue, Sep 24, 2024 at 9:02 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 24, 2024 5:05 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Thank you for considering another idea.
Thanks for reviewing the idea!
On Fri, Sep 20, 2024 at 2:46 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another ideathat can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce theperformance
impact. Thanks a lot to Amit for off-list discussion.
The concept of the new idea is that, the dead tuples are only useful to
detect
conflicts when applying *concurrent* transactions from remotes. Any
subsequent
UPDATE from a remote node after removing the dead tuples should have a
later
timestamp, meaning it's reasonable to detect an update_missing scenario
and
convert the UPDATE to an INSERT when applying it.
To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin afterconfirming
that all the concurrent transaction on publisher has been applied locally.
The replication slot used for this purpose will be a physical one or
logical one? And IIUC such a slot doesn't need to retain WAL but if we
do that, how do we advance the LSN of the slot?I think it would be a logical slot. We can keep the
restart_lsn/confirmed_flush_lsn as invalid because we don't need to retain the
WALs for decoding purpose.
As per my understanding, one of the main reasons to keep it logical is
to allow syncing it to standbys (slotsync functionality). It is
required because after promotion the subscriptions replicated to
standby could be enabled to make it a subscriber. If that is not
possible due to any reason then we can consider it to be a physical
slot as well.
--
With Regards,
Amit Kapila.
On Mon, Sep 23, 2024 at 8:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 24, 2024 5:05 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Thank you for considering another idea.
Thanks for reviewing the idea!
On Fri, Sep 20, 2024 at 2:46 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another ideathat can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce theperformance
impact. Thanks a lot to Amit for off-list discussion.
The concept of the new idea is that, the dead tuples are only useful to
detect
conflicts when applying *concurrent* transactions from remotes. Any
subsequent
UPDATE from a remote node after removing the dead tuples should have a
later
timestamp, meaning it's reasonable to detect an update_missing scenario
and
convert the UPDATE to an INSERT when applying it.
To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin afterconfirming
that all the concurrent transaction on publisher has been applied locally.
The replication slot used for this purpose will be a physical one or
logical one? And IIUC such a slot doesn't need to retain WAL but if we
do that, how do we advance the LSN of the slot?I think it would be a logical slot. We can keep the
restart_lsn/confirmed_flush_lsn as invalid because we don't need to retain the
WALs for decoding purpose.2) the apply worker send a new message to walsender to request the latest
wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedbackmessage or
extend the existing keepalive message(e,g extends the requestReply bit in
keepalive message to add a 'request_wal_position' value)The apply worker sends a keepalive message when it didn't receive
anything more than wal_receiver_timeout / 2. So in a very active
system, we cannot rely on piggybacking new information to the
keepalive messages to get the latest remote flush LSN.Right. I think we need to send this new message at some interval independent of
wal_receiver_timeout.3) The apply worker can continue to apply changes. After applying all the
WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all
concurrent
transactions have been applied. It can be effective for both bidirectional
and
non-bidirectional replication cases.
We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending todetect
update-delete conflicts should set retain_dead_tuples to true.
I'm still studying this idea but let me confirm the following scenario.
Suppose both Node-A and Node-B have the same row (1,1) in table t, and
XIDs and commit LSNs of T2 and T3 are the following:Node A
T2: DELETE FROM t WHERE id = 1 (10:02 AM) XID:100, commit-LSN:1000Node B
T3: UPDATE t SET value = 2 WHERE id 1 (10:01 AM) XID:500,
commit-LSN:5000Further suppose that it's now 10:05 AM, and the latest XID and the
latest flush WAL position of Node-A and Node-B are following:Node A
current XID: 300
latest flush LSN; 3000Node B
current XID: 700
latest flush LSN: 7000Both T2 and T3 are NOT sent to Node B and Node A yet, respectively
(i.e., the logical replication is delaying for 5 min).Consider the following scenario:
1. The apply worker on Node-A calls GetRunningTransactionData() and
gets 301 (set as candidate_xmin).
2. The apply worker on Node-A requests the latest WAL flush position
from Node-B, and gets 7000 (set as candidate_remote_wal_lsn).
3. T2 is applied on Node-B, and the latest flush position of Node-B is now 8000.
4. The apply worker on Node-A continues applying changes, and applies
the transactions up to remote (commit) LSN 7100.
5. Now that the apply worker on Node-A applied all changes smaller
than candidate_remote_wal_lsn (7000), it increases the slot.xmin to
301 (candidate_xmin).
6. On Node-A, vacuum runs and physically removes the tuple that was
deleted by T2.Here, on Node-B, there might be a transition between LSN 7100 and 8000
that might require the tuple that is deleted by T2.For example, "UPDATE t SET value = 3 WHERE id = 1" (say T4) is
executed on Node-B at LSN 7200, and it's sent to Node-A after step 6.
On Node-A, whether we detect "update_deleted" or "update_missing"
still depends on when vacuum removes the tuple deleted by T2.I think in this case, no matter we detect "update_delete" or "update_missing",
the final data is the same. Because T4's commit timestamp should be later than
T2 on node A, so in the case of "update_deleted", it will compare the commit
timestamp of the deleted tuple's xmax with T4's timestamp, and T4 should win,
which means we will convert the update into insert and apply. Even if the
deleted tuple is deleted and "update_missing" is detected, the update will
still be converted into insert and applied. So, the result is the same.
The "latest_timestamp_wins" is the default resolution method for
"update_deleted"? When I checked the wiki page[1]https://wiki.postgresql.org/wiki/Conflict_Detection_and_Resolution#Defaults, the "skip" was the
default solution method for that.
Regards,
[1]: https://wiki.postgresql.org/wiki/Conflict_Detection_and_Resolution#Defaults
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tuesday, September 24, 2024 2:42 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Sep 23, 2024 at 8:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, September 24, 2024 5:05 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I'm still studying this idea but let me confirm the following scenario.
Suppose both Node-A and Node-B have the same row (1,1) in table t,
and XIDs and commit LSNs of T2 and T3 are the following:Node A
T2: DELETE FROM t WHERE id = 1 (10:02 AM) XID:100,commit-LSN:1000
Node B
T3: UPDATE t SET value = 2 WHERE id 1 (10:01 AM) XID:500,
commit-LSN:5000Further suppose that it's now 10:05 AM, and the latest XID and the
latest flush WAL position of Node-A and Node-B are following:Node A
current XID: 300
latest flush LSN; 3000Node B
current XID: 700
latest flush LSN: 7000Both T2 and T3 are NOT sent to Node B and Node A yet, respectively
(i.e., the logical replication is delaying for 5 min).Consider the following scenario:
1. The apply worker on Node-A calls GetRunningTransactionData() and
gets 301 (set as candidate_xmin).
2. The apply worker on Node-A requests the latest WAL flush position
from Node-B, and gets 7000 (set as candidate_remote_wal_lsn).
3. T2 is applied on Node-B, and the latest flush position of Node-B is now8000.
4. The apply worker on Node-A continues applying changes, and
applies the transactions up to remote (commit) LSN 7100.
5. Now that the apply worker on Node-A applied all changes smaller
than candidate_remote_wal_lsn (7000), it increases the slot.xmin to
301 (candidate_xmin).
6. On Node-A, vacuum runs and physically removes the tuple that was
deleted by T2.Here, on Node-B, there might be a transition between LSN 7100 and
8000 that might require the tuple that is deleted by T2.For example, "UPDATE t SET value = 3 WHERE id = 1" (say T4) is
executed on Node-B at LSN 7200, and it's sent to Node-A after step 6.
On Node-A, whether we detect "update_deleted" or "update_missing"
still depends on when vacuum removes the tuple deleted by T2.I think in this case, no matter we detect "update_delete" or
"update_missing", the final data is the same. Because T4's commit
timestamp should be later than
T2 on node A, so in the case of "update_deleted", it will compare the
commit timestamp of the deleted tuple's xmax with T4's timestamp, and
T4 should win, which means we will convert the update into insert and
apply. Even if the deleted tuple is deleted and "update_missing" is
detected, the update will still be converted into insert and applied. So, theresult is the same.
The "latest_timestamp_wins" is the default resolution method for
"update_deleted"? When I checked the wiki page[1], the "skip" was the default
solution method for that.
Right, I think the wiki needs some update.
I think using 'skip' as default for update_delete could easily cause data
divergence when the dead tuple is deleted by an old transaction while the
UPDATE has a newer timestamp like the case you mentioned. It's necessary to
follow the last update win strategy when the incoming update has later
timestamp, which is to convert update to insert.
Best Regards,
Hou zj
On Tue, Sep 24, 2024 at 12:14 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 24, 2024 2:42 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Sep 23, 2024 at 8:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, September 24, 2024 5:05 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I'm still studying this idea but let me confirm the following scenario.
Suppose both Node-A and Node-B have the same row (1,1) in table t,
and XIDs and commit LSNs of T2 and T3 are the following:Node A
T2: DELETE FROM t WHERE id = 1 (10:02 AM) XID:100,commit-LSN:1000
Node B
T3: UPDATE t SET value = 2 WHERE id 1 (10:01 AM) XID:500,
commit-LSN:5000Further suppose that it's now 10:05 AM, and the latest XID and the
latest flush WAL position of Node-A and Node-B are following:Node A
current XID: 300
latest flush LSN; 3000Node B
current XID: 700
latest flush LSN: 7000Both T2 and T3 are NOT sent to Node B and Node A yet, respectively
(i.e., the logical replication is delaying for 5 min).Consider the following scenario:
1. The apply worker on Node-A calls GetRunningTransactionData() and
gets 301 (set as candidate_xmin).
2. The apply worker on Node-A requests the latest WAL flush position
from Node-B, and gets 7000 (set as candidate_remote_wal_lsn).
3. T2 is applied on Node-B, and the latest flush position of Node-B is now8000.
4. The apply worker on Node-A continues applying changes, and
applies the transactions up to remote (commit) LSN 7100.
5. Now that the apply worker on Node-A applied all changes smaller
than candidate_remote_wal_lsn (7000), it increases the slot.xmin to
301 (candidate_xmin).
6. On Node-A, vacuum runs and physically removes the tuple that was
deleted by T2.Here, on Node-B, there might be a transition between LSN 7100 and
8000 that might require the tuple that is deleted by T2.For example, "UPDATE t SET value = 3 WHERE id = 1" (say T4) is
executed on Node-B at LSN 7200, and it's sent to Node-A after step 6.
On Node-A, whether we detect "update_deleted" or "update_missing"
still depends on when vacuum removes the tuple deleted by T2.I think in this case, no matter we detect "update_delete" or
"update_missing", the final data is the same. Because T4's commit
timestamp should be later than
T2 on node A, so in the case of "update_deleted", it will compare the
commit timestamp of the deleted tuple's xmax with T4's timestamp, and
T4 should win, which means we will convert the update into insert and
apply. Even if the deleted tuple is deleted and "update_missing" is
detected, the update will still be converted into insert and applied. So, theresult is the same.
The "latest_timestamp_wins" is the default resolution method for
"update_deleted"? When I checked the wiki page[1], the "skip" was the default
solution method for that.Right, I think the wiki needs some update.
I think using 'skip' as default for update_delete could easily cause data
divergence when the dead tuple is deleted by an old transaction while the
UPDATE has a newer timestamp like the case you mentioned. It's necessary to
follow the last update win strategy when the incoming update has later
timestamp, which is to convert update to insert.
Right. If "latest_timestamp_wins" is the default resolution for
"update_deleted", I think your idea works fine unless I'm missing
corner cases.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Fri, Sep 20, 2024 at 2:46 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another idea that can
reliably detect conflicts without requiring users to choose a wise value for
the vacuum_committs_age. This new idea could also reduce the performance
impact. Thanks a lot to Amit for off-list discussion.The concept of the new idea is that, the dead tuples are only useful to detect
conflicts when applying *concurrent* transactions from remotes. Any subsequent
UPDATE from a remote node after removing the dead tuples should have a later
timestamp, meaning it's reasonable to detect an update_missing scenario and
convert the UPDATE to an INSERT when applying it.To achieve above, we can create an additional replication slot on the
subscriber side, maintained by the apply worker. This slot is used to retain
the dead tuples. The apply worker will advance the slot.xmin after confirming
that all the concurrent transaction on publisher has been applied locally.The process of advancing the slot.xmin could be:
1) the apply worker call GetRunningTransactionData() to get the
'oldestRunningXid' and consider this as 'candidate_xmin'.
2) the apply worker send a new message to walsender to request the latest wal
flush position(GetFlushRecPtr) on publisher, and save it to
'candidate_remote_wal_lsn'. Here we could introduce a new feedback message or
extend the existing keepalive message(e,g extends the requestReply bit in
keepalive message to add a 'request_wal_position' value)
3) The apply worker can continue to apply changes. After applying all the WALs
upto 'candidate_remote_wal_lsn', the apply worker can then advance the
slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It can be effective for both bidirectional and
non-bidirectional replication cases.We could introduce a boolean subscription option (retain_dead_tuples) to
control whether this feature is enabled. Each subscription intending to detect
update-delete conflicts should set retain_dead_tuples to true.As each apply worker needs a separate slot to retain deleted rows, the
requirement for slots will increase. The other possibility is to
maintain one slot by launcher or some other central process that
traverses all subscriptions, remember the ones marked with
retain_dead_rows (let's call this list as retain_sub_list). Then using
running_transactions get the oldest running_xact, and then get the
remote flush location from the other node (publisher node) and store
those as candidate values (candidate_xmin and
candidate_remote_wal_lsn) in slot. We can probably reuse existing
candidate variables of the slot. Next, we can check the remote_flush
locations from all the origins corresponding in retain_sub_list and if
all are ahead of candidate_remote_wal_lsn, we can update the slot's
xmin to candidate_xmin.
Yeah, I think that such an idea to reduce the number required slots
would be necessary.
I think in the above idea we can an optimization to combine the
request for remote wal LSN from different subscriptions pointing to
the same node to avoid sending multiple requests to the same node. I
am not sure if using pg_subscription.subconninfo is sufficient for
this, if not we can probably leave this optimization.If this idea is feasible then it would reduce the number of slots
required to retain the deleted rows but the launcher needs to get the
remote wal location corresponding to each publisher node. There are
two ways to achieve that (a) launcher requests one of the apply
workers corresponding to subscriptions pointing to the same publisher
node to get this information; (b) launcher launches another worker to
get the remote wal flush location.
I think the remote wal flush location is asked using a replication
protocol. Therefore, if a new worker is responsible for asking wal
flush location from multiple publishers (like the idea (b)), the
corresponding process would need to be launched on publisher sides and
logical replication would also need to start on each connection. I
think it would be better to get the remote wal flush location using
the existing logical replication connection (i.e., between the logical
wal sender and the apply worker), and advertise the locations on the
shared memory. Then, the central process who holds the slot to retain
the deleted row versions traverses them and increases slot.xmin if
possible.
The cost of requesting the remote wal flush location would not be huge
if we don't ask it very frequently. So probably we can start by having
each apply worker (in the retain_sub_list) ask the remote wal flush
location and can leave the optimization of avoiding sending the
request for the same publisher.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Friday, September 20, 2024 11:59 AM Hou, Zhijie/侯 志杰 wrote:
On Friday, September 20, 2024 10:55 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Friday, September 20, 2024 2:49 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I think that such a time-based configuration parameter would be a
reasonable solution. The current concerns are that it might affect
vacuum performance and lead to a similar bug we had withvacuum_defer_cleanup_age.
Thanks for the feedback!
I am working on the POC patch and doing some initial performance tests
on this idea.
I will share the results after finishing.
Here is a POC patch for vacuum_committs_age idea. The patch adds a GUC
vacuum_committs_age to prevent dead rows from being removed if the age of the
delete transaction (xmax) has not exceeded the vacuum_committs_age threshold.
E.g. , it ensures the row is retained if now() - commit_timestamp_of_xmax <
vacuum_committs_age.
However, please note that the patch is still unfinished due to a few
issues that need to be addressed. For instance: We need to prevent
relfrozenxid/datfrozenxid from being advanced in both aggressive and
non-aggressive vacuum modes. Otherwise, the commit timestamp data is cleaned
up after advancing frozenxid, and we won’t be able to compute the age of a tuple.
Additionally, the patch has a noticeable performance impact on vacuum
operations when rows in a table are deleted by multiple transactions. Here are
the results of VACUUMing a table after deleting each row in a separate
transaction (total of 10000000 dead rows) and the xmax ages of all the dead
tuples have exceeded the vacuum_committs_age in patched tests (see attachment
for the basic configuration of the tests):
HEAD: Time: 848.637 ms
patched, SLRU 8MB: Time: 1423.915 ms
patched, SLRU 1G: Time: 1310.869 ms
Since we have discussed about an alternative approach that can reliably retain
dead tuples without modifying vacuum process. We plan to shift our focus to
this new approach [1]/messages/by-id/CAD21AoD=m-YHceYMpsdu0HnGCaezeyVhaCPFxDLHU7aN0wgzqg@mail.gmail.com. I am currently working on another POC patch based on this
new approach and will share it later.
[1]: /messages/by-id/CAD21AoD=m-YHceYMpsdu0HnGCaezeyVhaCPFxDLHU7aN0wgzqg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
0001-try-to-add-vacuum_committs_age.patchapplication/octet-stream; name=0001-try-to-add-vacuum_committs_age.patchDownload
From 580a569694a28e72b75c85031ef149e3397b006f Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Fri, 13 Sep 2024 07:52:40 +0000
Subject: [PATCH] XXX: try to add vacuum_committs_age
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
The patch adds a GUC vacuum_committs_age to prevent dead rows from being
removed if the age of the delete transaction (xmax) has not exceeded the
vacuum_committs_age threshold. E.g. , it ensures the row is retained if
now() - commit_timestamp_of_xmax < vacuum_committs_age.
XXX please note that the patch is still unfinished due to a few
challenging issues that need to be addressed. For instance: We need to prevent
relfrozenxid/datfrozenxid from being advanced in both aggressive and
non-aggressive vacuum modes. Otherwise, the commit timestamp data is cleaned
up, and we won’t be able to compute the age of a tuple.
XXX the patch has a noticeable performance impact on vacuum
operations when rows in a table are deleted by multiple transactions.
---
src/backend/access/heap/heapam.c | 12 ++++++++--
src/backend/access/heap/pruneheap.c | 32 ++++++++++++++++++++++----
src/backend/access/transam/commit_ts.c | 25 ++++++++++++++++++++
src/backend/commands/vacuum.c | 1 +
src/backend/utils/misc/guc_tables.c | 11 +++++++++
src/include/access/heapam.h | 1 +
src/include/commands/vacuum.h | 1 +
7 files changed, 76 insertions(+), 7 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index f167107257..804c53b6ca 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -6531,6 +6531,7 @@ FreezeMultiXactId(MultiXactId multi, uint16 t_infomask,
bool
heap_prepare_freeze_tuple(HeapTupleHeader tuple,
const struct VacuumCutoffs *cutoffs,
+ bool xmax_aged,
HeapPageFreeze *pagefrz,
HeapTupleFreeze *frz, bool *totally_frozen)
{
@@ -6593,7 +6594,14 @@ heap_prepare_freeze_tuple(HeapTupleHeader tuple,
/* Now process xmax */
xid = frz->xmax;
- if (tuple->t_infomask & HEAP_XMAX_IS_MULTI)
+ if (!xmax_aged)
+ {
+ /*
+ * Skip freezing if the commit timestamp of xmax is within the
+ * vacuum_committs_age threshold.
+ */
+ }
+ else if (tuple->t_infomask & HEAP_XMAX_IS_MULTI)
{
/* Raw xmax is a MultiXactId */
TransactionId newxmax;
@@ -6900,7 +6908,7 @@ heap_freeze_tuple(HeapTupleHeader tuple,
pagefrz.NoFreezePageRelfrozenXid = FreezeLimit;
pagefrz.NoFreezePageRelminMxid = MultiXactCutoff;
- do_freeze = heap_prepare_freeze_tuple(tuple, &cutoffs,
+ do_freeze = heap_prepare_freeze_tuple(tuple, &cutoffs, false,
&pagefrz, &frz, &totally_frozen);
/*
diff --git a/src/backend/access/heap/pruneheap.c b/src/backend/access/heap/pruneheap.c
index 869d82ad66..b5afe671bf 100644
--- a/src/backend/access/heap/pruneheap.c
+++ b/src/backend/access/heap/pruneheap.c
@@ -14,6 +14,7 @@
*/
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/heapam.h"
#include "access/heapam_xlog.h"
#include "access/htup_details.h"
@@ -97,6 +98,8 @@ typedef struct
*/
int8 htsv[MaxHeapTuplesPerPage + 1];
+ bool commit_ts_aged[MaxHeapTuplesPerPage + 1];
+
/*
* Freezing-related state.
*/
@@ -155,7 +158,9 @@ typedef struct
/* Local functions */
static HTSV_Result heap_prune_satisfies_vacuum(PruneState *prstate,
HeapTuple tup,
- Buffer buffer);
+ Buffer buffer,
+ TimestampTz now,
+ bool *commit_ts_aged);
static inline HTSV_Result htsv_get_valid_status(int status);
static void heap_prune_chain(Page page, BlockNumber blockno, OffsetNumber maxoff,
OffsetNumber rootoffnum, PruneState *prstate);
@@ -367,6 +372,7 @@ heap_page_prune_and_freeze(Relation relation, Buffer buffer,
bool do_prune;
bool do_hint;
bool hint_bit_fpi;
+ TimestampTz now;
int64 fpi_before = pgWalUsage.wal_fpi;
/* Copy parameters to prstate */
@@ -469,6 +475,8 @@ heap_page_prune_and_freeze(Relation relation, Buffer buffer,
maxoff = PageGetMaxOffsetNumber(page);
tup.t_tableOid = RelationGetRelid(relation);
+ now = GetCurrentTimestamp();
+
/*
* Determine HTSV for all tuples, and queue them up for processing as HOT
* chain roots or as heap-only items.
@@ -543,7 +551,8 @@ heap_page_prune_and_freeze(Relation relation, Buffer buffer,
ItemPointerSet(&tup.t_self, blockno, offnum);
prstate.htsv[offnum] = heap_prune_satisfies_vacuum(&prstate, &tup,
- buffer);
+ buffer, now,
+ &prstate.commit_ts_aged[offnum]);
if (!HeapTupleHeaderIsHeapOnly(htup))
prstate.root_items[prstate.nroot_items++] = offnum;
@@ -914,23 +923,35 @@ heap_page_prune_and_freeze(Relation relation, Buffer buffer,
* Perform visibility checks for heap pruning.
*/
static HTSV_Result
-heap_prune_satisfies_vacuum(PruneState *prstate, HeapTuple tup, Buffer buffer)
+heap_prune_satisfies_vacuum(PruneState *prstate, HeapTuple tup, Buffer buffer,
+ TimestampTz now, bool *commit_ts_aged)
{
HTSV_Result res;
TransactionId dead_after;
+ TimestampTz commit_ts = (int64) 0;
+
+ *commit_ts_aged = true;
res = HeapTupleSatisfiesVacuumHorizon(tup, buffer, &dead_after);
if (res != HEAPTUPLE_RECENTLY_DEAD)
return res;
+ if (track_commit_timestamp && vacuum_committs_age)
+ {
+ TransactionIdGetCommitTsData(dead_after, &commit_ts, NULL);
+
+ *commit_ts_aged = TimestampDifferenceExceeds(commit_ts, now,
+ vacuum_committs_age * 1000);
+ }
+
/*
* For VACUUM, we must be sure to prune tuples with xmax older than
* OldestXmin -- a visibility cutoff determined at the beginning of
* vacuuming the relation. OldestXmin is used for freezing determination
* and we cannot freeze dead tuples' xmaxes.
*/
- if (prstate->cutoffs &&
+ if (*commit_ts_aged && prstate->cutoffs &&
TransactionIdIsValid(prstate->cutoffs->OldestXmin) &&
NormalTransactionIdPrecedes(dead_after, prstate->cutoffs->OldestXmin))
return HEAPTUPLE_DEAD;
@@ -943,7 +964,7 @@ heap_prune_satisfies_vacuum(PruneState *prstate, HeapTuple tup, Buffer buffer)
* if the GlobalVisState has been updated since the beginning of vacuuming
* the relation.
*/
- if (GlobalVisTestIsRemovableXid(prstate->vistest, dead_after))
+ if (*commit_ts_aged && GlobalVisTestIsRemovableXid(prstate->vistest, dead_after))
return HEAPTUPLE_DEAD;
return res;
@@ -1482,6 +1503,7 @@ heap_prune_record_unchanged_lp_normal(Page page, PruneState *prstate, OffsetNumb
if ((heap_prepare_freeze_tuple(htup,
prstate->cutoffs,
+ prstate->commit_ts_aged[offnum],
&prstate->pagefrz,
&prstate->frozen[prstate->nfrozen],
&totally_frozen)))
diff --git a/src/backend/access/transam/commit_ts.c b/src/backend/access/transam/commit_ts.c
index 77e1899d7a..8b71bc3245 100644
--- a/src/backend/access/transam/commit_ts.c
+++ b/src/backend/access/transam/commit_ts.c
@@ -121,6 +121,10 @@ static void DeactivateCommitTs(void);
static void WriteZeroPageXlogRec(int64 pageno);
static void WriteTruncateXlogRec(int64 pageno, TransactionId oldestXid);
+static TransactionId cached_xid = InvalidTransactionId;
+static TimestampTz cached_ts = 0;
+static RepOriginId cached_nodeid = InvalidRepOriginId;
+
/*
* TransactionTreeSetCommitTsData
*
@@ -212,6 +216,9 @@ TransactionTreeSetCommitTsData(TransactionId xid, int nsubxids,
if (TransactionIdPrecedes(TransamVariables->newestCommitTsXid, newestXact))
TransamVariables->newestCommitTsXid = newestXact;
LWLockRelease(CommitTsLock);
+
+ cached_xid = xid;
+ cached_ts = timestamp;
}
/*
@@ -293,6 +300,14 @@ TransactionIdGetCommitTsData(TransactionId xid, TimestampTz *ts,
*nodeid = 0;
return false;
}
+ else if (xid == cached_xid)
+ {
+ *ts = cached_ts;
+ if (nodeid)
+ *nodeid = cached_nodeid;
+
+ return *ts != 0;
+ }
LWLockAcquire(CommitTsLock, LW_SHARED);
@@ -311,6 +326,11 @@ TransactionIdGetCommitTsData(TransactionId xid, TimestampTz *ts,
*nodeid = commitTsShared->dataLastCommit.nodeid;
LWLockRelease(CommitTsLock);
+
+ cached_xid = xid;
+ cached_ts = *ts;
+ cached_nodeid = commitTsShared->dataLastCommit.nodeid;
+
return *ts != 0;
}
@@ -345,6 +365,11 @@ TransactionIdGetCommitTsData(TransactionId xid, TimestampTz *ts,
*nodeid = entry.nodeid;
LWLockRelease(SimpleLruGetBankLock(CommitTsCtl, pageno));
+
+ cached_xid = xid;
+ cached_ts = *ts;
+ cached_nodeid = entry.nodeid;
+
return *ts != 0;
}
diff --git a/src/backend/commands/vacuum.c b/src/backend/commands/vacuum.c
index 7d8e9d2045..e96808e3dc 100644
--- a/src/backend/commands/vacuum.c
+++ b/src/backend/commands/vacuum.c
@@ -70,6 +70,7 @@ int vacuum_multixact_freeze_min_age;
int vacuum_multixact_freeze_table_age;
int vacuum_failsafe_age;
int vacuum_multixact_failsafe_age;
+int vacuum_committs_age;
/*
* Variables for cost-based vacuum delay. The defaults differ between
diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c
index 686309db58..239e413a55 100644
--- a/src/backend/utils/misc/guc_tables.c
+++ b/src/backend/utils/misc/guc_tables.c
@@ -3714,6 +3714,17 @@ struct config_int ConfigureNamesInt[] =
NULL, NULL, NULL
},
+ {
+ {"vacuum_committs_age", PGC_SIGHUP, RESOURCES_DISK,
+ gettext_noop("Age at which VACUUM should *not* cleanup the dead tuple"),
+ NULL,
+ GUC_UNIT_S
+ },
+ &vacuum_committs_age,
+ 0, 0, 86400,
+ NULL, NULL, NULL
+ },
+
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, 0, 0, 0, NULL, NULL, NULL
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index b92eb506ec..846faa1595 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -340,6 +340,7 @@ extern TM_Result heap_lock_tuple(Relation relation, HeapTuple tuple,
extern void heap_inplace_update(Relation relation, HeapTuple tuple);
extern bool heap_prepare_freeze_tuple(HeapTupleHeader tuple,
const struct VacuumCutoffs *cutoffs,
+ bool xmax_aged,
HeapPageFreeze *pagefrz,
HeapTupleFreeze *frz, bool *totally_frozen);
diff --git a/src/include/commands/vacuum.h b/src/include/commands/vacuum.h
index 759f9a87d3..fadedeec28 100644
--- a/src/include/commands/vacuum.h
+++ b/src/include/commands/vacuum.h
@@ -296,6 +296,7 @@ extern PGDLLIMPORT int vacuum_multixact_freeze_min_age;
extern PGDLLIMPORT int vacuum_multixact_freeze_table_age;
extern PGDLLIMPORT int vacuum_failsafe_age;
extern PGDLLIMPORT int vacuum_multixact_failsafe_age;
+extern PGDLLIMPORT int vacuum_committs_age;
/*
* Maximum value for default_statistics_target and per-column statistics
--
2.30.0.windows.2
On Wednesday, September 25, 2024 2:23 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Sep 20, 2024 at 2:46 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 20, 2024 at 8:25 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Apart from the vacuum_defer_cleanup_age idea.
I think you meant to say vacuum_committs_age idea.
we’ve given more thought to our
approach for retaining dead tuples and have come up with another
idea that can reliably detect conflicts without requiring users to
choose a wise value for the vacuum_committs_age. This new idea could
also reduce the performance impact. Thanks a lot to Amit for off-listdiscussion.
The concept of the new idea is that, the dead tuples are only useful
to detect conflicts when applying *concurrent* transactions from
remotes. Any subsequent UPDATE from a remote node after removing the
dead tuples should have a later timestamp, meaning it's reasonable
to detect an update_missing scenario and convert the UPDATE to anINSERT when applying it.
To achieve above, we can create an additional replication slot on
the subscriber side, maintained by the apply worker. This slot is
used to retain the dead tuples. The apply worker will advance the
slot.xmin after confirming that all the concurrent transaction on publisherhas been applied locally.
The process of advancing the slot.xmin could be:
1) the apply worker call GetRunningTransactionData() to get the
'oldestRunningXid' and consider this as 'candidate_xmin'.
2) the apply worker send a new message to walsender to request the
latest wal flush position(GetFlushRecPtr) on publisher, and save it
to 'candidate_remote_wal_lsn'. Here we could introduce a new
feedback message or extend the existing keepalive message(e,g
extends the requestReply bit in keepalive message to add a
'request_wal_position' value)
3) The apply worker can continue to apply changes. After applying
all the WALs upto 'candidate_remote_wal_lsn', the apply worker can
then advance the slot.xmin to 'candidate_xmin'.This approach ensures that dead tuples are not removed until all
concurrent transactions have been applied. It can be effective for
both bidirectional and non-bidirectional replication cases.We could introduce a boolean subscription option
(retain_dead_tuples) to control whether this feature is enabled.
Each subscription intending to detect update-delete conflicts should setretain_dead_tuples to true.
As each apply worker needs a separate slot to retain deleted rows, the
requirement for slots will increase. The other possibility is to
maintain one slot by launcher or some other central process that
traverses all subscriptions, remember the ones marked with
retain_dead_rows (let's call this list as retain_sub_list). Then using
running_transactions get the oldest running_xact, and then get the
remote flush location from the other node (publisher node) and store
those as candidate values (candidate_xmin and
candidate_remote_wal_lsn) in slot. We can probably reuse existing
candidate variables of the slot. Next, we can check the remote_flush
locations from all the origins corresponding in retain_sub_list and if
all are ahead of candidate_remote_wal_lsn, we can update the slot's
xmin to candidate_xmin.Yeah, I think that such an idea to reduce the number required slots would be
necessary.I think in the above idea we can an optimization to combine the
request for remote wal LSN from different subscriptions pointing to
the same node to avoid sending multiple requests to the same node. I
am not sure if using pg_subscription.subconninfo is sufficient for
this, if not we can probably leave this optimization.If this idea is feasible then it would reduce the number of slots
required to retain the deleted rows but the launcher needs to get the
remote wal location corresponding to each publisher node. There are
two ways to achieve that (a) launcher requests one of the apply
workers corresponding to subscriptions pointing to the same publisher
node to get this information; (b) launcher launches another worker to
get the remote wal flush location.I think the remote wal flush location is asked using a replication protocol.
Therefore, if a new worker is responsible for asking wal flush location from
multiple publishers (like the idea (b)), the corresponding process would need
to be launched on publisher sides and logical replication would also need to
start on each connection. I think it would be better to get the remote wal flush
location using the existing logical replication connection (i.e., between the
logical wal sender and the apply worker), and advertise the locations on the
shared memory. Then, the central process who holds the slot to retain the
deleted row versions traverses them and increases slot.xmin if possible.The cost of requesting the remote wal flush location would not be huge if we
don't ask it very frequently. So probably we can start by having each apply
worker (in the retain_sub_list) ask the remote wal flush location and can leave
the optimization of avoiding sending the request for the same publisher.
Agreed. Here is the POC patch set based on this idea.
The implementation is as follows:
A subscription option is added to allow users to specify whether dead
tuples on the subscriber, which are 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
dead tuples from being removed. Note that if multiple subscriptions on one node
enable this option, only one replication slot will be created.
This additional slot will be used to retain dead tuples. Each apply worker will
maintain its own non-removable transaction ID by following the steps:
1) Calling GetRunningTransactionData() to take oldestRunningXid as the
candidate xid and send a new message to request the remote WAL position from
the walsender.
2) It then waits (non-blocking) to receive the WAL position from the walsender.
3) After receiving the WAL position, the non-removable transaction ID is
advanced if the current flush location has reached or surpassed the received
WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact. It ensures that dead tuples are not
removed until all concurrent transactions have been applied.
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.
I will keep testing the patch internally and analyze whether it's necessary to enable
failover for this new replication slot.
Please refer to the commit message of V2-0001 for the overall design.
The patch set is split into some parts to make it easier for the initial
review. Please note that each patch is interdependent and cannot work
independently.
Best Regards,
Hou zj
Attachments:
v2-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v2-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From 4333d357b0c53af4337f3083fe1c40202998d15a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v2 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 152 ++++++++++++++++++
2 files changed, 153 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..f2710e543f
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,152 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# Create table on node_A
+$node_A->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# Create the same table on node_B
+$node_B->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# 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_dead_tuples = 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_dead_tuples = 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");
+
+###############################################################################
+# 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);");
+$node_A->wait_for_catchup($subname_BA);
+
+my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+is($result, qq(1), '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 a=a+1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab;");
+
+$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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_missing.*\n.*DETAIL:.* Could not find the row to be deleted.*\n.*Replica identity \(a\)=\(1\)/,
+ 'delete target row is missing 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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the retain_dead_tuples option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v2-0002-Maintain-the-oldest-non-removeable-tranasction-id.patchapplication/octet-stream; name=v2-0002-Maintain-the-oldest-non-removeable-tranasction-id.patchDownload
From cc780560d69268cad6ec465cb12eb2f709938ff2 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v2 2/5] Maintain the oldest non removeable tranasction id by
apply worker
This patch allows each apply worker to maintain the oldest_nonremovable_xid in
the shared memory. The xid is advanced based on the following process:
1) Calling GetRunningTransactionData() to take oldestRunningXid as the
candidate xid and send a new message to request the remote WAL position from
the walsender.
2) It then waits (non-blocking) to receive the WAL position from the walsender.
3) After receiving the WAL position, the non-removable transaction ID is
advanced if the current flush location has reached or surpassed the received
WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
---
doc/src/sgml/protocol.sgml | 52 +++++++
src/backend/replication/logical/worker.c | 182 +++++++++++++++++++++-
src/backend/replication/walsender.c | 23 +++
src/include/replication/worker_internal.h | 18 +++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 275 insertions(+), 1 deletion(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 11b6456779..4896086220 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2440,6 +2440,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-wal-status-update">
+ <term>Primary WAL status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary WAL status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2584,6 +2619,23 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Standby WAL status request (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('x')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for the WAL status on the primary.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..d74d8fe2de 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,8 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +395,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3592,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3715,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's')
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3737,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3838,7 +3871,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;
@@ -3916,6 +3948,154 @@ 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-delete 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:
+ *
+ * DTR_REQUEST_WALSENDER_WAL_POS - Call GetRunningTransactionData() to get the
+ * candidate xmin and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_WALSENDER_WAL_POS - Wait for receiving the WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_LOCAL_FLUSH - Advance the non-removable transaction ID if the
+ * current flush location has reached or surpassed the received WAL position.
+ *
+ * Retaining the dead tuples for this period is sufficient because any
+ * subsequent transaction from the publisher will have a later timestamp.
+ * Therefore, it is acceptable if dead tuples are removed by vacuum and an
+ * update_missing conflict is detected, as the correct resolution for the
+ * last-update-wins strategy in this case is to convert the UPDATE to an INSERT
+ * and apply it anyway.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attemp_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_wal_pos);
+
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->retaindeadtuples)
+ return;
+
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attemp_time);
+
+ /*
+ * Return if we have requested but not yet received the remote wal
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_wal_pos))
+ return;
+
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_wal_pos) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *remote_wal_pos)
+ return;
+
+ /*
+ * Advance the non-removable transaction id if the remote wal position
+ * has been received, and all transactions up to that position on the
+ * publisher have been applied and flushed locally.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ RunningTransactions running_transactions;
+ TransactionId oldest_running_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attemp_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attemp_time = now;
+
+ running_transactions = GetRunningTransactionData();
+ oldest_running_xid = running_transactions->oldestRunningXid;
+ epoch = EpochFromFullTransactionId(TransamVariables->nextXid);
+
+ /* Release the locks acquired in GetRunningTransactionData() */
+ LWLockRelease(ProcArrayLock);
+ LWLockRelease(XidGenLock);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > running_transactions->nextXid)
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ elog(DEBUG2, "sending wal position request message");
+
+ /* Send a wal position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "x", sizeof(uint8));
+
+ *remote_wal_pos = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 c5f1009f37..5371711488 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2342,6 +2343,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'x':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2688,6 +2693,24 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the standby message requesting the latest WAL write position.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ elog(DEBUG2, "sending wal write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, GetXLogWriteRecPtr());
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5fabb127d7..66d7f16a73 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v2-0001-add-a-retain_dead_tuples-option-to-subscriptions.patchapplication/octet-stream; name=v2-0001-add-a-retain_dead_tuples-option-to-subscriptions.patchDownload
From 0cb21ed97274ffd823ca259ea5687e9414e5fe3f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:01:42 +0800
Subject: [PATCH v2 1/5] Add a retain_dead_tuples option to subscriptions
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.
We assume that the appropriate resolution for update_deleted conflicts, to
achieve eventual consistency, is the latest_timestamp_wins 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.
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) Calling GetRunningTransactionData() to take oldestRunningXid as the
candidate xid and send a new message to request the remote WAL position from
the walsender.
2) It then waits (non-blocking) to receive the WAL position from the walsender.
3) After receiving the WAL position, the non-removable transaction ID is
advanced if the current flush location has reached or surpassed the received
WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples from being removed. Note that if multiple subscriptions on one node
enable this option, only one replication slot will be created.
The actual replication slot management is implemented in the following patches.
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 17 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.c | 10 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 184 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
13 files changed, 227 insertions(+), 90 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bfb97865e1..8d5e48d65c 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8046,6 +8046,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretaindeadtuples</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, dead tuples on the subscriber that are still useful for
+ detecting <xref-linkend="conflict-update-deleted"/> are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..1660e99f5b 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 740b7d9421..140753c474 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -428,6 +428,23 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-dead-tuples">
+ <term><literal>retain_dead_tuples</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether dead tuples on the subscriber that are still useful
+ for detecting <xref linkend="conflict-update-deleted"/> should be
+ retained. The default is <literal>false</literal>. If set to true, the
+ detection of <xref linkend="conflict-update-deleted"/> will be
+ enabled, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 9efc9159f2..c66ed565b5 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -72,6 +72,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->passwordrequired = subform->subpasswordrequired;
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
+ sub->retaindeadtuples = subform->subretaindeadtuples;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 49109dbdc8..0cfdf6d276 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1357,7 +1357,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)
+ subretaindeadtuples, 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 02ccc636b8..9206e3f486 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_DEAD_TUPLES 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 retaindeadtuples;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DEAD_TUPLES))
+ opts->retaindeadtuples = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,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_DEAD_TUPLES) &&
+ strcmp(defel->defname, "retain_dead_tuples") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
+ opts->retaindeadtuples = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -594,7 +608,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_DEAD_TUPLES | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -701,6 +716,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_subretaindeadtuples - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1196,7 +1213,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1356,6 +1373,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
+ {
+ values[Anum_pg_subscription_subretaindeadtuples - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 130b80775d..bd91ae2821 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4847,6 +4847,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretaindeadtuples;
int i,
ntups;
@@ -4919,11 +4920,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretaindeadtuples\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretaindeadtuples\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4962,6 +4969,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5008,6 +5016,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subretaindeadtuples =
+ pg_strdup(PQgetvalue(res, i, i_subretaindeadtuples));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5248,6 +5258,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subretaindeadtuples, "t") == 0)
+ appendPQExpBufferStr(query, ", retain_dead_tuples = 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 9f907ed5ad..4b44360580 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -673,6 +673,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subretaindeadtuples;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 6a36c91083..506f6d9f25 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6547,7 +6547,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)
{
@@ -6615,6 +6615,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subretaindeadtuples AS \"%s\"\n",
+ gettext_noop("Retain dead tuples"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a7ccde6d7d..d29f42cd1a 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1947,8 +1947,9 @@ psql_completion(const char *text, int start, int end)
/* ALTER SUBSCRIPTION <name> SET ( */
else if (HeadMatches("ALTER", "SUBSCRIPTION", MatchAny) && TailMatches("SET", "("))
COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "run_as_owner", "slot_name",
- "streaming", "synchronous_commit", "two_phase");
+ "password_required", "retain_dead_tuples",
+ "run_as_owner", "slot_name", "streaming",
+ "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (HeadMatches("ALTER", "SUBSCRIPTION", MatchAny) && TailMatches("SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3344,8 +3345,9 @@ psql_completion(const char *text, int start, int end)
else if (HeadMatches("CREATE", "SUBSCRIPTION") && TailMatches("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_dead_tuples",
+ "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 0aa14ec4a2..8653f2beee 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subretaindeadtuples; /* True if retaining dead tuples for
+ * conflict detection is required */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 retaindeadtuples; /* True if retaining dead tuples for
+ * conflict detection is required */
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 17d48b1685..f489c58312 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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | 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 | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | 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_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 | 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 dead tuples | 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)
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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | off | 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 | 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 dead tuples | 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;
@@ -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 dead tuples | 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 dead tuples | 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 dead tuples | 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 dead tuples | 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 dead tuples | 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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 dead tuples | 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 dead tuples | 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,50 @@ 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 | 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 dead tuples | 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)
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 | off | 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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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_dead_tuples must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = foo);
+ERROR: retain_dead_tuples requires a Boolean value
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = true);
+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 dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (retain_dead_tuples = 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 dead tuples | 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)
+
+ALTER SUBSCRIPTION regress_testsub SET (retain_dead_tuples = true);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | 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 007c9e7037..405f63bf03 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - retain_dead_tuples must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = foo);
+
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = true);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (retain_dead_tuples = false);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (retain_dead_tuples = true);
+
+\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;
--
2.30.0.windows.2
v2-0003-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v2-0003-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From 7da24074e9476f6d1601ada815d99dfd97551ff8 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v2 3/5] 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 if any local subscription has the
retain_dead_tuples option enabled.
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.
---
src/backend/replication/logical/launcher.c | 217 +++++++++++++++++++++
1 file changed, 217 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..4eaff836ff 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,14 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+#define CONFLICT_DETECTION_SLOT "pg_conflict_detection"
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +108,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -157,6 +168,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->retaindeadtuples = subform->subretaindeadtuples;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1127,6 +1139,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1170,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool retain_dead_tuples = false;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1174,11 +1191,28 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->retaindeadtuples;
+
if (!sub->enabled)
continue;
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1249,31 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (retain_dead_tuples)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1242,6 +1301,164 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ bool updated_xmin = false;
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_PERSISTENT, false, false, false);
+
+ LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+ xmin_horizon = GetOldestSafeDecodingTransactionId(true);
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = xmin_horizon;
+ MyReplicationSlot->data.xmin = xmin_horizon;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ ReplicationSlotsComputeRequiredXmin(true);
+
+ LWLockRelease(ProcArrayLock);
+ }
+ }
+
+ /* No need to update xmin if the slot has been invalidated */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ /*
+ * Only a WARNING is reported here, which is intended to avoid
+ * preventing the launcher from starting logical replication workers.
+ */
+ ereport(WARNING,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("The replication slot \"%s\" has been invalidated",
+ CONFLICT_DETECTION_SLOT),
+ errhint("Drop the replication slot \"%s\".",
+ CONFLICT_DETECTION_SLOT));
+
+ return false;
+ }
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+
+ if (FullTransactionIdIsValid(new_xmin) &&
+ FullTransactionIdPrecedes(full_xmin, new_xmin))
+ {
+ updated_xmin = true;
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
+ }
+
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ if (!updated_xmin)
+ return false;
+
+ /* 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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?
*/
--
2.30.0.windows.2
v2-0004-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v2-0004-Support-the-conflict-detection-for-update_deleted.patchDownload
From e5dddbf760ffd92371b4db45a4ac3ccb0f430d64 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v2 4/5] 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 | 13 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 140 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 55 +++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 98a7ad0c27..63223a637f 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1637,6 +1637,19 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>detect_conflict</literal></link>
+ are enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d83e99da49..513b70ccfe 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 0cfdf6d276..624a9687bb 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..5d2e025c89 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,9 @@ tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
if (att->attisdropped || att->attgenerated)
continue;
+ if (columns && !bms_is_member(att->attnum, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +429,7 @@ retry:
/* 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;
@@ -474,6 +480,132 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * Search the relation 'rel' for the most recently deleted tuple that matches
+ * the values in 'searchslot' and is not yet removable by VACUUM.
+ *
+ * The commit timestamp of the transaction that deleted the tuple is used to
+ * determine whether the tuple is the most recently deleted one.
+ *
+ * This operation can be quite slow on tables with a large number of rows.
+ * However, it is primarily used in rare conflict cases where the target row
+ * for an update cannot be found.
+ */
+bool
+FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ bool found = false;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+
+ if (OidIsValid(localindexoid))
+ indexbitmap = build_index_column_bitmap(localindexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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;
+ found = true;
+ }
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return found;
+}
+
/*
* 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 d74d8fe2de..e2d03cfecf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2688,6 +2688,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2704,15 +2707,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2723,7 +2722,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2742,19 +2741,29 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(localrel);
+
+ if (MySubscription->retaindeadtuples &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3076,7 +3085,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3086,17 +3095,27 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(partrel);
+
+ if (MySubscription->retaindeadtuples &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3108,7 +3127,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3119,7 +3138,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 17b0fc02ef..2ee2fb69e4 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1972,7 +1972,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2000,11 +2000,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 05fcbf7515..a420c554b4 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5590,9 +5590,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..bb04ef86c2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index f5434d8365..1c85992ace 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,11 +2144,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
On Mon, Sep 30, 2024 at 12:02 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wednesday, September 25, 2024 2:23 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I think the remote wal flush location is asked using a replication protocol.
Therefore, if a new worker is responsible for asking wal flush location from
multiple publishers (like the idea (b)), the corresponding process would need
to be launched on publisher sides and logical replication would also need to
start on each connection. I think it would be better to get the remote wal flush
location using the existing logical replication connection (i.e., between the
logical wal sender and the apply worker), and advertise the locations on the
shared memory. Then, the central process who holds the slot to retain the
deleted row versions traverses them and increases slot.xmin if possible.The cost of requesting the remote wal flush location would not be huge if we
don't ask it very frequently. So probably we can start by having each apply
worker (in the retain_sub_list) ask the remote wal flush location and can leave
the optimization of avoiding sending the request for the same publisher.Agreed. Here is the POC patch set based on this idea.
The implementation is as follows:
A subscription option is added to allow users to specify whether dead
tuples on the subscriber, which are 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,
I find the option name retain_dead_tuples bit misleading because by
name one can't make out the purpose of the same. It is better to name
it as detect_update_deleted or something on those lines.
and an additional replication
slot named pg_conflict_detection will be created on the subscriber to prevent
dead tuples from being removed. Note that if multiple subscriptions on one node
enable this option, only one replication slot will be created.
In general, we should have done this by default but as detecting
update_deleted type conflict has some overhead in terms of retaining
dead tuples for more time, so having an option seems reasonable. But I
suggest to keep this as a separate last patch. If we can make the core
idea work by default then we can enable it via option in the end.
--
With Regards,
Amit Kapila.
On Tuesday, October 1, 2024 8:44 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Sep 30, 2024 at 12:02 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Wednesday, September 25, 2024 2:23 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I think the remote wal flush location is asked using a replication protocol.
Therefore, if a new worker is responsible for asking wal flush
location from multiple publishers (like the idea (b)), the
corresponding process would need to be launched on publisher sides
and logical replication would also need to start on each connection.
I think it would be better to get the remote wal flush location
using the existing logical replication connection (i.e., between the
logical wal sender and the apply worker), and advertise the
locations on the shared memory. Then, the central process who holds theslot to retain the deleted row versions traverses them and increases slot.xmin if
possible.The cost of requesting the remote wal flush location would not be
huge if we don't ask it very frequently. So probably we can start by
having each apply worker (in the retain_sub_list) ask the remote wal
flush location and can leave the optimization of avoiding sending therequest for the same publisher.
Agreed. Here is the POC patch set based on this idea.
The implementation is as follows:
A subscription option is added to allow users to specify whether dead
tuples on the subscriber, which are 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,I find the option name retain_dead_tuples bit misleading because by name one
can't make out the purpose of the same. It is better to name it as
detect_update_deleted or something on those lines.and an additional replication
slot named pg_conflict_detection will be created on the subscriber to
prevent dead tuples from being removed. Note that if multiple
subscriptions on one node enable this option, only one replication slot will becreated.
In general, we should have done this by default but as detecting
update_deleted type conflict has some overhead in terms of retaining dead
tuples for more time, so having an option seems reasonable. But I suggest to
keep this as a separate last patch. If we can make the core idea work by default
then we can enable it via option in the end.
Thanks for the comments. I have renamed the option to detect_update_deleted and
make it a separate last patch in the V3 patch set.
Best Regards,
Hou zj
Attachments:
v3-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v3-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From a2f6df14163fec84dbcf4d7342e12d6ce901216d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v3 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 152 ++++++++++++++++++
2 files changed, 153 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..9815ed7618
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,152 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# Create table on node_A
+$node_A->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# Create the same table on node_B
+$node_B->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);");
+$node_A->wait_for_catchup($subname_BA);
+
+my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+is($result, qq(1), '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 a=a+1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab;");
+
+$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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_missing.*\n.*DETAIL:.* Could not find the row to be deleted.*\n.*Replica identity \(a\)=\(1\)/,
+ 'delete target row is missing 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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v3-0001-Maintain-the-oldest-non-removeable-tranasction-id.patchapplication/octet-stream; name=v3-0001-Maintain-the-oldest-non-removeable-tranasction-id.patchDownload
From 19c52d18345688f9ae179c7ccc21005dcfd23fca Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v3 1/5] 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.
We assume that the appropriate resolution for update_deleted conflicts, to
achieve eventual consistency, is the latest_timestamp_wins 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.
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) Calling GetRunningTransactionData() to take oldestRunningXid as the
candidate xid and send a new message to request the remote WAL position from
the walsender.
2) It then waits (non-blocking) to receive the WAL position from the walsender.
3) After receiving the WAL position, the non-removable transaction ID is
advanced if the current flush location has reached or surpassed the received
WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 52 ++++++
src/backend/replication/logical/worker.c | 185 +++++++++++++++++++++-
src/backend/replication/walsender.c | 23 +++
src/include/replication/worker_internal.h | 18 +++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 278 insertions(+), 1 deletion(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 2d2481bb8b..fd69b7b9cf 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2440,6 +2440,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-wal-status-update">
+ <term>Primary WAL status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary WAL status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2584,6 +2619,23 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Standby WAL status request (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('x')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for the WAL status on the primary.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..a55ae55e1b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,8 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +395,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3592,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3715,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's')
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3737,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3838,7 +3871,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;
@@ -3916,6 +3948,157 @@ 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-delete 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:
+ *
+ * DTR_REQUEST_WALSENDER_WAL_POS - Call GetRunningTransactionData() to get the
+ * candidate xmin and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_WALSENDER_WAL_POS - Wait for receiving the WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_LOCAL_FLUSH - Advance the non-removable transaction ID if the
+ * current flush location has reached or surpassed the received WAL position.
+ *
+ * Retaining the dead tuples for this period is sufficient because any
+ * subsequent transaction from the publisher will have a later timestamp.
+ * Therefore, it is acceptable if dead tuples are removed by vacuum and an
+ * update_missing conflict is detected, as the correct resolution for the
+ * last-update-wins strategy in this case is to convert the UPDATE to an INSERT
+ * and apply it anyway.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attemp_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_wal_pos);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attemp_time);
+
+ /*
+ * Return if we have requested but not yet received the remote wal
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_wal_pos))
+ return;
+
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_wal_pos) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *remote_wal_pos)
+ return;
+
+ /*
+ * Advance the non-removable transaction id if the remote wal position
+ * has been received, and all transactions up to that position on the
+ * publisher have been applied and flushed locally.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ RunningTransactions running_transactions;
+ TransactionId oldest_running_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attemp_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attemp_time = now;
+
+ running_transactions = GetRunningTransactionData();
+ oldest_running_xid = running_transactions->oldestRunningXid;
+ epoch = EpochFromFullTransactionId(TransamVariables->nextXid);
+
+ /* Release the locks acquired in GetRunningTransactionData() */
+ LWLockRelease(ProcArrayLock);
+ LWLockRelease(XidGenLock);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > running_transactions->nextXid)
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ elog(DEBUG2, "sending wal position request message");
+
+ /* Send a wal position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "x", sizeof(uint8));
+
+ *remote_wal_pos = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 c5f1009f37..5371711488 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2342,6 +2343,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'x':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2688,6 +2693,24 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the standby message requesting the latest WAL write position.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ elog(DEBUG2, "sending wal write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, GetXLogWriteRecPtr());
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 5fabb127d7..66d7f16a73 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v3-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v3-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From 7ed5f4fcda21ca338746fb8fac772aa0e4d0466d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v3 2/5] 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.
---
src/backend/replication/logical/launcher.c | 209 +++++++++++++++++++++
1 file changed, 209 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..7acdc1ebdd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,14 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+#define CONFLICT_DETECTION_SLOT "pg_conflict_detection"
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +108,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1138,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1169,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool updated;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1179,6 +1195,21 @@ ApplyLauncherMain(Datum main_arg)
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1246,26 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ if (sublist)
+ {
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ slot_maybe_exist = true;
+ }
+ 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. */
@@ -1242,6 +1293,164 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ bool updated_xmin = false;
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_PERSISTENT, false, false, false);
+
+ LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
+
+ xmin_horizon = GetOldestSafeDecodingTransactionId(true);
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = xmin_horizon;
+ MyReplicationSlot->data.xmin = xmin_horizon;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ ReplicationSlotsComputeRequiredXmin(true);
+
+ LWLockRelease(ProcArrayLock);
+ }
+ }
+
+ /* No need to update xmin if the slot has been invalidated */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ /*
+ * Only a WARNING is reported here, which is intended to avoid
+ * preventing the launcher from starting logical replication workers.
+ */
+ ereport(WARNING,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("The replication slot \"%s\" has been invalidated",
+ CONFLICT_DETECTION_SLOT),
+ errhint("Drop the replication slot \"%s\".",
+ CONFLICT_DETECTION_SLOT));
+
+ return false;
+ }
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+
+ if (FullTransactionIdIsValid(new_xmin) &&
+ FullTransactionIdPrecedes(full_xmin, new_xmin))
+ {
+ updated_xmin = true;
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
+ }
+
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ if (!updated_xmin)
+ return false;
+
+ /* 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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?
*/
--
2.30.0.windows.2
v3-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v3-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 83c0a67042a682a7c54fdd94e513ea38461fd048 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v3 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 140 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 53 +++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 235 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 98a7ad0c27..c44b0e8108 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1637,6 +1637,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 48ffe87241..510092ef73 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 49109dbdc8..d1e5b9de8b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1370,6 +1370,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..5d2e025c89 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,9 @@ tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
if (att->attisdropped || att->attgenerated)
continue;
+ if (columns && !bms_is_member(att->attnum, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +429,7 @@ retry:
/* 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;
@@ -474,6 +480,132 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * Search the relation 'rel' for the most recently deleted tuple that matches
+ * the values in 'searchslot' and is not yet removable by VACUUM.
+ *
+ * The commit timestamp of the transaction that deleted the tuple is used to
+ * determine whether the tuple is the most recently deleted one.
+ *
+ * This operation can be quite slow on tables with a large number of rows.
+ * However, it is primarily used in rare conflict cases where the target row
+ * for an update cannot be found.
+ */
+bool
+FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ bool found = false;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+
+ if (OidIsValid(localindexoid))
+ indexbitmap = build_index_column_bitmap(localindexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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;
+ found = true;
+ }
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return found;
+}
+
/*
* 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 a55ae55e1b..ec301f8f21 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2688,6 +2688,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2704,15 +2707,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2723,7 +2722,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2742,19 +2741,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(localrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3076,7 +3084,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3086,17 +3094,26 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(partrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3108,7 +3125,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3119,7 +3136,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 17b0fc02ef..2ee2fb69e4 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1972,7 +1972,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2000,11 +2000,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 05fcbf7515..a420c554b4 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5590,9 +5590,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..bb04ef86c2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index f5434d8365..1c85992ace 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,11 +2144,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v3-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v3-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From 799a8d5c0aadb4f3313a9f60541718c9599a1855 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 2 Oct 2024 10:33:25 +0800
Subject: [PATCH v3 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 17 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 13 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.c | 10 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 184 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 252 insertions(+), 106 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bfb97865e1..c4b1c6abad 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8046,6 +8046,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c44b0e8108..7402e21536 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1644,7 +1644,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_conflict</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 740b7d9421..0b1b6e5a62 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -428,6 +428,23 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 9efc9159f2..849af83f1f 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -72,6 +72,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->passwordrequired = subform->subpasswordrequired;
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index d1e5b9de8b..1f6dfb398e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1357,7 +1357,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)
+ subdetectupdatedeleted, 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 02ccc636b8..b8ef44fae9 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -594,7 +608,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -701,6 +716,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1196,7 +1213,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1356,6 +1373,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 7acdc1ebdd..a04492330c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -168,6 +168,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,7 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool updated;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1190,6 +1191,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
continue;
@@ -1246,24 +1249,29 @@ ApplyLauncherMain(Datum main_arg)
}
}
- if (sublist)
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (retain_dead_tuples)
{
- /*
- * Maintain the xmin value of the replication slot for conflict
- * detection if needed, and update the sleep time before the next
- * attempt.
- */
- updated = advance_conflict_slot_xmin(xmin);
+ bool updated = advance_conflict_slot_xmin(xmin);
compute_slot_update_naptime(updated, &slot_update_wait_time);
wait_time = Min(wait_time, slot_update_wait_time);
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;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ec301f8f21..05fcd3b9f6 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2745,7 +2745,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(localrel);
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -3098,7 +3099,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(partrel);
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -4004,11 +4006,8 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
Assert(remote_wal_pos);
- /*
- * The non-removable transaction ID for a subscription is centrally
- * managed by the main apply worker.
- */
- if (!am_leader_apply_worker())
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
return;
if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 130b80775d..a39cda21d2 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4847,6 +4847,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4919,11 +4920,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4962,6 +4969,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5008,6 +5016,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5248,6 +5258,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9f907ed5ad..bd29448060 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -673,6 +673,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 6a36c91083..313ea2120c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6547,7 +6547,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)
{
@@ -6615,6 +6615,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/bin/psql/tab-complete.c b/src/bin/psql/tab-complete.c
index a7ccde6d7d..d78f35f62b 100644
--- a/src/bin/psql/tab-complete.c
+++ b/src/bin/psql/tab-complete.c
@@ -1947,8 +1947,9 @@ psql_completion(const char *text, int start, int end)
/* ALTER SUBSCRIPTION <name> SET ( */
else if (HeadMatches("ALTER", "SUBSCRIPTION", MatchAny) && TailMatches("SET", "("))
COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "run_as_owner", "slot_name",
- "streaming", "synchronous_commit", "two_phase");
+ "password_required", "detect_update_deleted",
+ "run_as_owner", "slot_name", "streaming",
+ "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (HeadMatches("ALTER", "SUBSCRIPTION", MatchAny) && TailMatches("SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3344,8 +3345,9 @@ psql_completion(const char *text, int start, int end)
else if (HeadMatches("CREATE", "SUBSCRIPTION") && TailMatches("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", "detect_update_deleted",
+ "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 0aa14ec4a2..6f141df950 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 17d48b1685..52df143112 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | 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_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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | off | 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 | 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 | Detect update deleted | 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;
@@ -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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 | Detect update deleted | 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 | Detect update deleted | 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,50 @@ 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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = false);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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)
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | 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 007c9e7037..94c9aacd6f 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = false);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
Dear Hou,
Thanks for updating the patch! Here are my comments.
My comments do not take care which file contains the change, and the ordering may
be random.
1.
```
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_conflict</literal></link>
+ are enabled.
```
"detect_conflict" still exists, it should be "detect_update_deleted".
2. maybe_advance_nonremovable_xid
```
+ /* Send a wal position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "x", sizeof(uint8))
```
I think the character is used for PoC purpose, so it's about time we change it.
How about:
- 'W', because it requests the WAL location, or
- 'S', because it is accosiated with 's' message.
3. maybe_advance_nonremovable_xid
```
+ if (!AllTablesyncsReady())
+ return;
```
If we do not update oldest_nonremovable_xid during the sync, why do we send
the status message? I feel we can return in any phases if !AllTablesyncsReady().
4. advance_conflict_slot_xmin
```
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_PERSISTENT, false, false, false);
```
Hmm. You said the slot would be logical, but now it is physical. Which is correct?
5. advance_conflict_slot_xmin
```
+ xmin_horizon = GetOldestSafeDecodingTransactionId(true);
```
Since the slot won't do the logical decoding, we do not have to use oldest-safe-decoding
xid. I feel it is OK to use the latest xid.
6. advance_conflict_slot_xmin
```
+ /* No need to update xmin if the slot has been invalidated */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
```
I feel the slot won't be invalidated. According to
InvalidatePossiblyObsoleteSlot(), the physical slot cannot be invalidated if it
has invalid restart_lsn.
7. ApplyLauncherMain
```
+ retain_dead_tuples |= sub->detectupdatedeleted;
```
Can you tell me why it must be updated even if the sub is disabled?
8. ApplyLauncherMain
If the subscription which detect_update_deleted = true exists but wal_receiver_status_interval = 0,
the slot won't be advanced and dead tuple retains forever... is it right? Can we
avoid it anyway?
9. FindMostRecentlyDeletedTupleInfo
It looks for me that the scan does not use indexes even if exists, but I feel it should use.
Am I missing something or is there a reason?
[1]: /messages/by-id/OS0PR01MB5716E0A283D1B66954CDF5A694682@OS0PR01MB5716.jpnprd01.prod.outlook.com
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Wednesday, October 2, 2024 2:34 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch! Here are my comments.
My comments do not take care which file contains the change, and the ordering
may be random.
Thanks for the comments!
1.
```
+ and <link
linkend="sql-createsubscription-params-with-detect-update-deleted"><lite
ral>detect_conflict</literal></link>
+ are enabled.
```
"detect_conflict" still exists, it should be "detect_update_deleted".
Fixed.
2. maybe_advance_nonremovable_xid ``` + /* Send a wal position request message to the server */ + walrcv_send(LogRepWorkerWalRcvConn, "x", sizeof(uint8)) ``` I think the character is used for PoC purpose, so it's about time we change it. How about:- 'W', because it requests the WAL location, or
- 'S', because it is accosiated with 's' message.
Thanks for the suggestions. I preferred 'W'.
3. maybe_advance_nonremovable_xid
```
+ if (!AllTablesyncsReady())
+ return;
```
If we do not update oldest_nonremovable_xid during the sync, why do we send
the status message? I feel we can return in any phases
if !AllTablesyncsReady().
It's possible the table sync start before sending the request and stop before
entering DTR_WAIT_FOR_LOCAL_FLUSH phase, in which case we would be able to
advance xid immediately. So, I think there is no harm to send the request when
table sync is in progress and it could increase the possibility of advancing
the xid.
4. advance_conflict_slot_xmin ``` + ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, + RS_PERSISTENT, false, false, false); ``` Hmm. You said the slot would be logical, but now it is physical. Which is correct?
The statement (slot would be logical) was true for the original design, which
is to let each apply worker maintain a different replication slot. But now
since we have decided to have the launcher maintain a single slot, it would be
unnecessary to make it logical one.
5. advance_conflict_slot_xmin
```
+ xmin_horizon = GetOldestSafeDecodingTransactionId(true);
```
Since the slot won't do the logical decoding, we do not have to use
oldest-safe-decoding xid. I feel it is OK to use the latest xid.
I think using latest xid(nextXid) means that we would ignore all the running transactions.
E.g. All the dead tuples deleted by current running transactions will not be
retained. Since the apply worker chooses the oldest running xid as the
non-removable xid, so it's natrual for the launcher to be consistent by taking
the running transactions into account(use the lowest xid not affected by vacuum
as the initial xmin).
6. advance_conflict_slot_xmin ``` + /* No need to update xmin if the slot has been invalidated */ + if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE) ``` I feel the slot won't be invalidated. According to InvalidatePossiblyObsoleteSlot(), the physical slot cannot be invalidated if it has invalid restart_lsn.
I was considering it as a sanity check. And it seems possible for user to drop
this slot and create a slot with the same name, in which case we may face the
invalidation and it maybe helpful to report some message in this case.
But after thinking again, it seems better to just disallow user from using
this reserved replication slot name and remove this check, so changed like
that.
7. ApplyLauncherMain
```
+ retain_dead_tuples |= sub->detectupdatedeleted;
```
Can you tell me why it must be updated even if the sub is disabled?
I think we won't update the xmin if all the subscriptions are disabled.
Because the launcher cannot get a valid xmin to update.
But when checking this part, I found a bug where it could update the xmin
When some of the subscriptions are disabled, and fixed it in this version.
8. ApplyLauncherMain
If the subscription which detect_update_deleted = true exists but
wal_receiver_status_interval = 0, the slot won't be advanced and dead tuple
retains forever... is it right? Can we avoid it anyway?
I think the same is true for the hot_standby_feedback feature. We can try to
avoid it but not sure if it's worth it. Anyway, user have options to either adjust
wal_receiver_status_interval or disable detect_update_deleted in this case.
9. FindMostRecentlyDeletedTupleInfo
It looks for me that the scan does not use indexes even if exists, but I feel it
should use.
Am I missing something or is there a reason?
I missed that it's possible to use the index to fetch dead tuples. I have added it
In this version. Thanks a lot to Kuroda-san for contributing codes off-list.
[1]:
/messages/by-id/OS0PR01MB5716E0A283D1B66954
CDF5A694682%40OS0PR01MB5716.jpnprd01.prod.outlook.com
Attach the V4 patch set which addressed above comments.
Best Regards,
Hou zj
Attachments:
v4-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v4-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From 01cee9a9ce68cc1658f76838a55502d962baab8d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v4 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 152 ++++++++++++++++++
2 files changed, 153 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..9815ed7618
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,152 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# Create table on node_A
+$node_A->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# Create the same table on node_B
+$node_B->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);");
+$node_A->wait_for_catchup($subname_BA);
+
+my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+is($result, qq(1), '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 a=a+1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab;");
+
+$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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_missing.*\n.*DETAIL:.* Could not find the row to be deleted.*\n.*Replica identity \(a\)=\(1\)/,
+ 'delete target row is missing 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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v4-0001-Maintain-the-oldest-non-removeable-tranasction-id.patchapplication/octet-stream; name=v4-0001-Maintain-the-oldest-non-removeable-tranasction-id.patchDownload
From 036a15d9180e180d2704078a001d88b2f9138353 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v4 1/4] 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.
We assume that the appropriate resolution for update_deleted conflicts, to
achieve eventual consistency, is the latest_timestamp_wins 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.
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) Calling GetRunningTransactionData() to take oldestRunningXid as the
candidate xid and send a new message to request the remote WAL position from
the walsender.
2) It then waits (non-blocking) to receive the WAL position from the walsender.
3) After receiving the WAL position, the non-removable transaction ID is
advanced if the current flush location has reached or surpassed the received
WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 52 +++++++
src/backend/replication/logical/worker.c | 180 +++++++++++++++++++++-
src/backend/replication/walsender.c | 23 +++
src/include/replication/worker_internal.h | 18 +++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 273 insertions(+), 1 deletion(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 057c46f3f5..92fe145d60 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2440,6 +2440,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-wal-status-update">
+ <term>Primary WAL status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary WAL status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2584,6 +2619,23 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Standby WAL status request (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('W')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for the WAL status on the primary.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..d61da33e3f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,8 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +395,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3592,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3715,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's')
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3737,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3838,7 +3871,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;
@@ -3916,6 +3948,152 @@ 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-delete 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:
+ *
+ * DTR_REQUEST_WALSENDER_WAL_POS - Call GetRunningTransactionData() to get the
+ * candidate xmin and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_WALSENDER_WAL_POS - Wait for receiving the WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_LOCAL_FLUSH - Advance the non-removable transaction ID if the
+ * current flush location has reached or surpassed the received WAL position.
+ *
+ * Retaining the dead tuples for this period is sufficient because any
+ * subsequent transaction from the publisher will have a later timestamp.
+ * Therefore, it is acceptable if dead tuples are removed by vacuum and an
+ * update_missing conflict is detected, as the correct resolution for the
+ * last-update-wins strategy in this case is to convert the UPDATE to an INSERT
+ * and apply it anyway.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attemp_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_wal_pos);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attemp_time);
+
+ /*
+ * Return if we have requested but not yet received the remote wal
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_wal_pos))
+ return;
+
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_wal_pos) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *remote_wal_pos)
+ return;
+
+ /*
+ * Advance the non-removable transaction id if the remote wal position
+ * has been received, and all transactions up to that position on the
+ * publisher have been applied and flushed locally.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xix;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attemp_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attemp_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xix = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xix);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xix))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ elog(DEBUG2, "sending wal position request message");
+
+ /* Send a wal position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "W", sizeof(uint8));
+
+ *remote_wal_pos = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 371eef3ddd..919a790a35 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'W':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,24 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the standby message requesting the latest WAL write position.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ elog(DEBUG2, "sending wal write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, GetXLogWriteRecPtr());
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index a65e1c07c5..c332bcec7b 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v4-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v4-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From f2022ee70c7ecaa2d6970cd54e08b24a62b49067 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v4 2/4] 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.
---
src/backend/replication/logical/launcher.c | 195 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 222 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..c31d4dcedf 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1248,29 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1298,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index c7bfbb15e0..15a0f8ce58 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -42,6 +42,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -128,6 +130,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 919a790a35..e57f68d1be 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v4-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v4-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 85abec97c03d7366bfbc89807418e5d1a6d99538 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v4 3/4] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 155 +++++++++++++++++++++
src/backend/replication/logical/conflict.c | 22 +++
src/backend/replication/logical/worker.c | 53 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 254 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 98a7ad0c27..c44b0e8108 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1637,6 +1637,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 331315f8d3..e2cd5ec07f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3456b821bc..2e306ceff6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..4876366a22 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -474,6 +477,158 @@ retry:
return found;
}
+/*
+ * If the tuple is identified as dead and was deleted by a transaction with a
+ * more recent commit timestamp, update the transaction ID, deletion time, and
+ * origin information associated with this tuple.
+ */
+static void
+update_recent_dead_tuple_info(TupleTableSlot *scanslot,
+ TransactionId oldestXmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ bool dead = false;
+ TransactionId xmax;
+ TimestampTz localts;
+ RepOriginId localorigin;
+
+ hslot = (BufferHeapTupleTableSlot *) scanslot;
+
+ tuple = ExecFetchSlotHeapTuple(scanslot, false, NULL);
+ buf = hslot->buffer;
+
+ LockBuffer(buf, BUFFER_LOCK_SHARE);
+
+ if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!dead)
+ return;
+
+ xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data);
+ if (!TransactionIdIsValid(xmax))
+ return;
+
+ /* 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;
+ }
+}
+
+/*
+ * 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.
+ *
+ * 'localindexoid' should be the replica identity index used to locate the
+ * target row for update or deletion operations
+ *
+ * If 'localindexoid' is invalid, the function defaults to performing a full
+ * table scan, which can be slow on tables with a large number of rows.
+ * However, it is primarily used in rare conflict cases where the target row
+ * for an update cannot be found.
+ */
+bool
+FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ scanslot = table_slot_create(rel, NULL);
+
+ /* Use local index as much as possible */
+ if (OidIsValid(localindexoid))
+ {
+ Relation idxrel;
+ ScanKeyData skey[INDEX_MAX_KEYS];
+ int skey_attoff;
+ IndexScanDesc scan;
+
+ idxrel = index_open(localindexoid, RowExclusiveLock);
+
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /* Start an index scan. */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, skey_attoff, 0);
+
+ index_rescan(scan, skey, skey_attoff, NULL, 0);
+
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ Assert(tuples_equal(scanslot, searchslot, eq));
+ update_recent_dead_tuple_info(scanslot, oldestXmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ index_endscan(scan);
+
+ index_close(idxrel, NoLock);
+ }
+ else
+ {
+ TableScanDesc scan;
+
+ /* Start a heap scan. */
+ scan = table_beginscan(rel, SnapshotAny, 0, NULL);
+
+ table_rescan(scan, NULL);
+
+ /* Try to find the tuple */
+ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot))
+ {
+ if (!tuples_equal(scanslot, searchslot, eq))
+ continue;
+
+ update_recent_dead_tuple_info(scanslot, oldestXmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 d61da33e3f..247f62b31c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2688,6 +2688,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2704,15 +2707,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2723,7 +2722,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2742,19 +2741,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(localrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3076,7 +3084,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3086,17 +3094,26 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(partrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3108,7 +3125,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3119,7 +3136,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index f7b50e0b5a..0a3353eb5c 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1978,7 +1978,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2006,11 +2006,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 3ae31a614c..b8d9c15d47 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5597,9 +5597,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..bb04ef86c2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2b47013f11..4dad6e4966 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2145,11 +2145,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v4-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v4-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From 5765dea33f2876f4d13e3ebcef3e78385b5544ae Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 11 Oct 2024 15:11:28 +0800
Subject: [PATCH v4 4/4] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 17 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 34 +++-
src/backend/replication/logical/launcher.c | 7 +-
src/backend/replication/logical/worker.c | 10 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 184 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
15 files changed, 240 insertions(+), 90 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 964c819a02..039bd33d89 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8046,6 +8046,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c44b0e8108..f657e56368 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1644,7 +1644,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 8a3096e62b..b0a04ba90a 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -428,6 +428,23 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 9efc9159f2..849af83f1f 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -72,6 +72,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->passwordrequired = subform->subpasswordrequired;
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2e306ceff6..c38881d61a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1358,7 +1358,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)
+ subdetectupdatedeleted, 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 02ccc636b8..5b06698fda 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -594,7 +608,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -701,6 +716,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1196,7 +1213,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1356,6 +1373,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ ApplyLauncherWakeupAtCommit();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c31d4dcedf..8b1b02a331 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,6 +1190,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
{
can_advance_xmin = false;
@@ -1253,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated = advance_conflict_slot_xmin(xmin);
compute_slot_update_naptime(updated, &slot_update_wait_time);
@@ -1269,6 +1273,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 247f62b31c..0bf0b02d70 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2745,7 +2745,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(localrel);
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -3098,7 +3099,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(partrel);
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -4004,6 +4006,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
Assert(remote_wal_pos);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1b47c388ce..078967e3a3 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4850,6 +4850,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4922,11 +4923,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4965,6 +4972,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5011,6 +5019,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5251,6 +5261,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9f907ed5ad..bd29448060 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -673,6 +673,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 6a36c91083..313ea2120c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6547,7 +6547,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)
{
@@ -6615,6 +6615,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 0aa14ec4a2..6f141df950 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 17d48b1685..52df143112 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | 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_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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | off | 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 | 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 | Detect update deleted | 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;
@@ -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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 | Detect update deleted | 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 | Detect update deleted | 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,50 @@ 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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = false);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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)
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | 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 007c9e7037..94c9aacd6f 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = false);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
On Friday, October 11, 2024 4:35 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
Attach the V4 patch set which addressed above comments.
While reviewing the patch, I noticed that the current design could not work in
a non-bidirectional cluster (publisher -> subscriber) when the publisher is
also a physical standby. (We supported logical decoding on a physical standby
recently, so it's possible to take a physical standby as a logical publisher).
The cluster looks like:
physical primary -> physical standby (also publisher) -> logical subscriber (detect_update_deleted)
The issue arises because the physical standby (acting as the publisher) might
lag behind its primary. As a result, the logical walsender on the standby might
not be able to get the latest WAL position when requested by the logical
subscriber. We 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 older
commit timestamp. (Note that transactions on both primary and standby have
the same commit timestamp).
So, the logical walsender might send an outdated WAL position as feedback.
This, in turn, can cause the replication slot on the subscriber to advance
prematurely, leading to the unwanted removal of dead tuples. As a result, the
apply worker may fail to correctly detect update-delete conflicts.
We thought of few options to fix this:
1) Add a Time-Based Subscription Option:
We could add a new time-based option for subscriptions, such as
retain_dead_tuples = '5s'. In the logical launcher, after obtaining the
candidate XID, the launcher will wait for the specified time before advancing
the slot.xmin. This ensures that deleted tuples are retained for at least the
duration defined by this new option.
This approach is designed to simulate the functionality of the GUC
(vacuum_committs_age), but with a simpler implementation that does not impact
vacuum performance. We can maintain both this time-based method and the current
automatic method. If a user does not specify the time-based option, we will
continue using the existing approach to retain dead tuples until all concurrent
transactions from the remote node have been applied.
2) Modification to the Logical Walsender
On the logical walsender, which is as a physical standby, we can build an
additional connection to the physical primary to obtain the latest WAL
position. This position will then be sent as feedback to the logical
subscriber.
A potential concern is that this requires the walsender to use the walreceiver
API, which may seem a bit unnatural. And, it starts an additional walsender
process on the primary, as the logical walsender on the physical standby will
need to communicate with this walsender to fetch the WAL position.
3) Documentation of Restrictions
As an alternative, we could simply document the restriction that detecting
update_delete is not supported if the publisher is also acting as a physical
standby.
Best Regards,
Hou zj
On Mon, Oct 14, 2024 at 9:09 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
While reviewing the patch, I noticed that the current design could not work in
a non-bidirectional cluster (publisher -> subscriber) when the publisher is
also a physical standby. (We supported logical decoding on a physical standby
recently, so it's possible to take a physical standby as a logical publisher).The cluster looks like:
physical primary -> physical standby (also publisher) -> logical subscriber (detect_update_deleted)
The issue arises because the physical standby (acting as the publisher) might
lag behind its primary. As a result, the logical walsender on the standby might
not be able to get the latest WAL position when requested by the logical
subscriber. We 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 older
commit timestamp. (Note that transactions on both primary and standby have
the same commit timestamp).So, the logical walsender might send an outdated WAL position as feedback.
This, in turn, can cause the replication slot on the subscriber to advance
prematurely, leading to the unwanted removal of dead tuples. As a result, the
apply worker may fail to correctly detect update-delete conflicts.We thought of few options to fix this:
1) Add a Time-Based Subscription Option:
We could add a new time-based option for subscriptions, such as
retain_dead_tuples = '5s'. In the logical launcher, after obtaining the
candidate XID, the launcher will wait for the specified time before advancing
the slot.xmin. This ensures that deleted tuples are retained for at least the
duration defined by this new option.This approach is designed to simulate the functionality of the GUC
(vacuum_committs_age), but with a simpler implementation that does not impact
vacuum performance. We can maintain both this time-based method and the current
automatic method. If a user does not specify the time-based option, we will
continue using the existing approach to retain dead tuples until all concurrent
transactions from the remote node have been applied.2) Modification to the Logical Walsender
On the logical walsender, which is as a physical standby, we can build an
additional connection to the physical primary to obtain the latest WAL
position. This position will then be sent as feedback to the logical
subscriber.A potential concern is that this requires the walsender to use the walreceiver
API, which may seem a bit unnatural. And, it starts an additional walsender
process on the primary, as the logical walsender on the physical standby will
need to communicate with this walsender to fetch the WAL position.
This idea is worth considering, but I think it may not be a good
approach if the physical standby is cascading. We need to restrict the
update_delete conflict detection, if the standby is cascading, right?
The other approach is that we send current_timestamp from the
subscriber and somehow check if the physical standby has applied
commit_lsn up to that commit_ts, if so, it can send that WAL position
to the subscriber, otherwise, wait for it to be applied. If we do this
then we don't need to add a restriction for cascaded physical standby.
I think the subscriber anyway needs to wait for such an LSN to be
applied on standby before advancing the xmin even if we get it from
the primary. This is because the subscriber can only be able to apply
and flush the WAL once it is applied on the standby. Am, I missing
something?
This approach has a disadvantage that we are relying on clocks to be
synced on both nodes which we anyway need for conflict resolution as
discussed in the thread [1]/messages/by-id/CABdArM4=152B9PoyF4kggQ4LniYtjBCdUjL=qBwD-jcogP2BPQ@mail.gmail.com. We also need to consider the Commit
Timestamp and LSN inversion issue as discussed in another thread [2]/messages/by-id/CAJpy0uBxEJnabEp3JS=n9X19Vx2ZK3k5AR7N0h-cSMtOwYV3fA@mail.gmail.com
if we want to pursue this approach because we may miss an LSN that has
a prior timestamp.
3) Documentation of Restrictions
As an alternative, we could simply document the restriction that detecting
update_delete is not supported if the publisher is also acting as a physical
standby.
If we don't want to go for something along the lines of the approach
mentioned in (2) then I think we can do a combination of (1) and (3)
where we can error out if the user has not provided retain_dead_tuples
and the publisher is physical standby.
[1]: /messages/by-id/CABdArM4=152B9PoyF4kggQ4LniYtjBCdUjL=qBwD-jcogP2BPQ@mail.gmail.com
[2]: /messages/by-id/CAJpy0uBxEJnabEp3JS=n9X19Vx2ZK3k5AR7N0h-cSMtOwYV3fA@mail.gmail.com
--
With Regards,
Amit Kapila.
On Tue, Oct 15, 2024 at 5:03 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Oct 14, 2024 at 9:09 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:We thought of few options to fix this:
1) Add a Time-Based Subscription Option:
We could add a new time-based option for subscriptions, such as
retain_dead_tuples = '5s'. In the logical launcher, after obtaining the
candidate XID, the launcher will wait for the specified time before advancing
the slot.xmin. This ensures that deleted tuples are retained for at least the
duration defined by this new option.This approach is designed to simulate the functionality of the GUC
(vacuum_committs_age), but with a simpler implementation that does not impact
vacuum performance. We can maintain both this time-based method and the current
automatic method. If a user does not specify the time-based option, we will
continue using the existing approach to retain dead tuples until all concurrent
transactions from the remote node have been applied.2) Modification to the Logical Walsender
On the logical walsender, which is as a physical standby, we can build an
additional connection to the physical primary to obtain the latest WAL
position. This position will then be sent as feedback to the logical
subscriber.A potential concern is that this requires the walsender to use the walreceiver
API, which may seem a bit unnatural. And, it starts an additional walsender
process on the primary, as the logical walsender on the physical standby will
need to communicate with this walsender to fetch the WAL position.This idea is worth considering, but I think it may not be a good
approach if the physical standby is cascading. We need to restrict the
update_delete conflict detection, if the standby is cascading, right?The other approach is that we send current_timestamp from the
subscriber and somehow check if the physical standby has applied
commit_lsn up to that commit_ts, if so, it can send that WAL position
to the subscriber, otherwise, wait for it to be applied. If we do this
then we don't need to add a restriction for cascaded physical standby.
I think the subscriber anyway needs to wait for such an LSN to be
applied on standby before advancing the xmin even if we get it from
the primary. This is because the subscriber can only be able to apply
and flush the WAL once it is applied on the standby. Am, I missing
something?This approach has a disadvantage that we are relying on clocks to be
synced on both nodes which we anyway need for conflict resolution as
discussed in the thread [1]. We also need to consider the Commit
Timestamp and LSN inversion issue as discussed in another thread [2]
if we want to pursue this approach because we may miss an LSN that has
a prior timestamp.
The problem due to Commit Timestamp and LSN inversion is that the
standby may not consider the WAL LSN from an earlier timestamp, which
could lead to the removal of required dead rows on the subscriber.
The other problem pointed out by Hou-San offlist due to Commit
Timestamp and LSN inversion is that we could miss sending the WAL LSN
that the subscriber requires to retain dead rows for update_delete
conflict. For example, consider the following case 2 node,
bidirectional setup:
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AM
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AM
Say subscription is created with retain_dead_tuples = true/false
After executing T2, the apply worker on Node A will check the latest
wal flush location on Node B. Till that time, the T3 should have
finished, so the xmin will be advanced only after applying the WALs
that is later than T3. So, the dead tuple will not be removed before
applying the T3, which means the update_delete can be detected.
As there is a gap between when we acquire the commit_timestamp and the
commit LSN, it is possible that T3 would have not yet flushed it's WAL
even though it is committed earlier than T2. If this happens then we
won't be able to detect update_deleted conflict reliably.
Now, the one simpler idea is to acquire the commit timestamp and
reserve WAL (LSN) under the same spinlock in
ReserveXLogInsertLocation() but that could be costly as discussed in
the thread [1]/messages/by-id/CAJpy0uBxEJnabEp3JS=n9X19Vx2ZK3k5AR7N0h-cSMtOwYV3fA@mail.gmail.com. The other more localized solution is to acquire a
timestamp after reserving the commit WAL LSN outside the lock which
will solve this particular problem.
[1]: /messages/by-id/CAJpy0uBxEJnabEp3JS=n9X19Vx2ZK3k5AR7N0h-cSMtOwYV3fA@mail.gmail.com
--
With Regards,
Amit Kapila.
On Fri, Oct 11, 2024 at 2:04 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V4 patch set which addressed above comments.
A few minor comments:
1.
+ * Retaining the dead tuples for this period is sufficient because any
+ * subsequent transaction from the publisher will have a later timestamp.
+ * Therefore, it is acceptable if dead tuples are removed by vacuum and an
+ * update_missing conflict is detected, as the correct resolution for the
+ * last-update-wins strategy in this case is to convert the UPDATE to an INSERT
+ * and apply it anyway.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase)
We should cover the key point of retaining dead tuples which is to
avoid converting updates to inserts (considering the conflict as
update_missing) in the comments above and also in the commit message.
2. In maybe_advance_nonremovable_xid() all three phases are handled by
different if blocks but as per my understanding the phase value will
be unique in one call to the function. So, shouldn't it be handled
with else if?
--
With Regards,
Amit Kapila.
On Friday, October 18, 2024 5:47 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Oct 11, 2024 at 2:04 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Attach the V4 patch set which addressed above comments.
A few minor comments:
Thanks for the comments.
1. + * Retaining the dead tuples for this period is sufficient because any + * subsequent transaction from the publisher will have a later timestamp. + * Therefore, it is acceptable if dead tuples are removed by vacuum and +an + * update_missing conflict is detected, as the correct resolution for +the + * last-update-wins strategy in this case is to convert the UPDATE to +an INSERT + * and apply it anyway. + * + * The 'remote_wal_pos' will be reset after sending a new request to walsender. + */ +static void +maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos, + DeadTupleRetainPhase *phase)We should cover the key point of retaining dead tuples which is to avoid
converting updates to inserts (considering the conflict as
update_missing) in the comments above and also in the commit message.
Right, I updated the comments the commit messages for the same.
2. In maybe_advance_nonremovable_xid() all three phases are handled by
different if blocks but as per my understanding the phase value will be unique
in one call to the function. So, shouldn't it be handled with else if?
I thought that it is possible that we can immediately pass the check in next phase,
thus, wanted to run the codes of next phase as well during one function call.
For example, after switching from DTR_WAIT_FOR_LOCAL_FLUSH to
DTR_REQUEST_WALSENDER_WAL_POS, it’s possible that enough time have passed since
the last wal position request, so It was intended to let it immediately send
the next request in the next if block if possible.
I added some comments in this version to make it clear.
Here is the V5 patch set which addressed above comments.
Best Regards,
Hou zj
Attachments:
v5-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v5-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From 5d4575b6c7c950e3a0b7ae9f81ce61576d930543 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v5 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 152 ++++++++++++++++++
2 files changed, 153 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..9815ed7618
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,152 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# Create table on node_A
+$node_A->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# Create the same table on node_B
+$node_B->safe_psql('postgres', "CREATE TABLE tab (a int PRIMARY KEY)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);");
+$node_A->wait_for_catchup($subname_BA);
+
+my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+is($result, qq(1), '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 a=a+1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab;");
+
+$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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_missing.*\n.*DETAIL:.* Could not find the row to be deleted.*\n.*Replica identity \(a\)=\(1\)/,
+ 'delete target row is missing 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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v5-0001-Maintain-the-oldest-non-removeable-tranasction-id.patchapplication/octet-stream; name=v5-0001-Maintain-the-oldest-non-removeable-tranasction-id.patchDownload
From e64e7cdabe8bf1d07d14db220598864332e53b24 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v5] 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. While for concurrent remote
transactions with earlier timestamps, detecting update_delete 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 latest_timestamp_wins 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) Calling GetOldestActiveTransactionId() to take oldestRunningXid as the
candidate xid and send a new message to request the remote WAL position from
the walsender.
2) It then waits (non-blocking) to receive the WAL position from the walsender.
3) After receiving the WAL position, the non-removable transaction ID is
advanced if the current flush location has reached or surpassed the received
WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 52 ++++++
src/backend/replication/logical/worker.c | 197 +++++++++++++++++++++-
src/backend/replication/walsender.c | 23 +++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 290 insertions(+), 1 deletion(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 057c46f3f5..92fe145d60 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2440,6 +2440,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-wal-status-update">
+ <term>Primary WAL status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary WAL status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2584,6 +2619,23 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Standby WAL status request (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('W')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for the WAL status on the primary.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..a6f4d41eb5 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,8 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +395,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3592,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3715,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's')
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3737,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3838,7 +3871,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;
@@ -3916,6 +3948,169 @@ 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-delete 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:
+ *
+ * DTR_REQUEST_WALSENDER_WAL_POS - Call GetOldestActiveTransactionId() to get
+ * the candidate xmin and send a message to request the remote WAL position
+ * from the walsender.
+ *
+ * DTR_WAIT_FOR_WALSENDER_WAL_POS - Wait for receiving the WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_LOCAL_FLUSH - Advance the non-removable transaction ID if the
+ * current flush location has reached or surpassed the received WAL position.
+ *
+ * Retaining the dead tuples for this period is sufficient for ensuring
+ * eventual consistency using last-update-wins strategy, which involves
+ * converting an UPDATE to an INSERT and applying it if remote transactions
+ * have later commit timestamps, while ignoring UPDATES with earlier
+ * timestamps. Since any subsequent transaction from the publisher after
+ * advancing the transaction ID will have a later timestamp, it is acceptable
+ * for dead tuples to be removed by VACUUM and an update_missing conflict is
+ * detected, as the correct resolution in this case is to convert apply it
+ * anyway. It's only necessary to detect dead tuples and compare their
+ * timestamps when applying transactions that occurred before advancing the
+ * non-removable transaction ID, as they might have earlier commit timestamps.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attemp_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_wal_pos);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attemp_time);
+
+ /*
+ * Return if we have requested but not yet received the remote wal
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_wal_pos))
+ return;
+
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+
+ /*
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_wal_pos. Proceeding to the next
+ * phase to check if we can immediately advance the transaction ID.
+ */
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_wal_pos) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *remote_wal_pos)
+ return;
+
+ /*
+ * Advance the non-removable transaction id if the remote wal position
+ * has been received, and all transactions up to that position on the
+ * publisher have been applied and flushed locally.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * wal position request. Proceeding to the next phase to determine if
+ * we can send the next request.
+ */
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xix;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attemp_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attemp_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xix = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xix);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xix))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ elog(DEBUG2, "sending wal position request message");
+
+ /* Send a wal position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "W", sizeof(uint8));
+
+ *remote_wal_pos = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 371eef3ddd..919a790a35 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'W':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,24 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the standby message requesting the latest WAL write position.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ elog(DEBUG2, "sending wal write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, GetXLogWriteRecPtr());
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 57de1acff3..5854ac9974 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v5-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v5-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From ccd7ec0b05d1af618d52f278928f172494d38e44 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v5 2/5] 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.
---
src/backend/replication/logical/launcher.c | 195 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 222 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..c31d4dcedf 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1248,29 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1298,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index c7bfbb15e0..15a0f8ce58 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -42,6 +42,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -128,6 +130,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 919a790a35..e57f68d1be 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v5-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v5-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 408a17cc7bb5d8661821d006fa916544b935240f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v5 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 155 +++++++++++++++++++++
src/backend/replication/logical/conflict.c | 22 +++
src/backend/replication/logical/worker.c | 53 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 254 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 98a7ad0c27..c44b0e8108 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1637,6 +1637,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 331315f8d3..e2cd5ec07f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3456b821bc..2e306ceff6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..4876366a22 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -474,6 +477,158 @@ retry:
return found;
}
+/*
+ * If the tuple is identified as dead and was deleted by a transaction with a
+ * more recent commit timestamp, update the transaction ID, deletion time, and
+ * origin information associated with this tuple.
+ */
+static void
+update_recent_dead_tuple_info(TupleTableSlot *scanslot,
+ TransactionId oldestXmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ bool dead = false;
+ TransactionId xmax;
+ TimestampTz localts;
+ RepOriginId localorigin;
+
+ hslot = (BufferHeapTupleTableSlot *) scanslot;
+
+ tuple = ExecFetchSlotHeapTuple(scanslot, false, NULL);
+ buf = hslot->buffer;
+
+ LockBuffer(buf, BUFFER_LOCK_SHARE);
+
+ if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!dead)
+ return;
+
+ xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data);
+ if (!TransactionIdIsValid(xmax))
+ return;
+
+ /* 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;
+ }
+}
+
+/*
+ * 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.
+ *
+ * 'localindexoid' should be the replica identity index used to locate the
+ * target row for update or deletion operations
+ *
+ * If 'localindexoid' is invalid, the function defaults to performing a full
+ * table scan, which can be slow on tables with a large number of rows.
+ * However, it is primarily used in rare conflict cases where the target row
+ * for an update cannot be found.
+ */
+bool
+FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ scanslot = table_slot_create(rel, NULL);
+
+ /* Use local index as much as possible */
+ if (OidIsValid(localindexoid))
+ {
+ Relation idxrel;
+ ScanKeyData skey[INDEX_MAX_KEYS];
+ int skey_attoff;
+ IndexScanDesc scan;
+
+ idxrel = index_open(localindexoid, RowExclusiveLock);
+
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /* Start an index scan. */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, skey_attoff, 0);
+
+ index_rescan(scan, skey, skey_attoff, NULL, 0);
+
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ Assert(tuples_equal(scanslot, searchslot, eq));
+ update_recent_dead_tuple_info(scanslot, oldestXmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ index_endscan(scan);
+
+ index_close(idxrel, NoLock);
+ }
+ else
+ {
+ TableScanDesc scan;
+
+ /* Start a heap scan. */
+ scan = table_beginscan(rel, SnapshotAny, 0, NULL);
+
+ table_rescan(scan, NULL);
+
+ /* Try to find the tuple */
+ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot))
+ {
+ if (!tuples_equal(scanslot, searchslot, eq))
+ continue;
+
+ update_recent_dead_tuple_info(scanslot, oldestXmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 5c69bd084a..c29b8f86c0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2688,6 +2688,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2704,15 +2707,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2723,7 +2722,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2742,19 +2741,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(localrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3076,7 +3084,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3086,17 +3094,26 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(partrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3108,7 +3125,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3119,7 +3136,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index f7b50e0b5a..0a3353eb5c 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1978,7 +1978,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2006,11 +2006,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 7c0b74fe05..18db2e466e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5597,9 +5597,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..bb04ef86c2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2b47013f11..4dad6e4966 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2145,11 +2145,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v5-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v5-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From 150315ed34fd291eed1484e123bab401defcc05e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 11 Oct 2024 15:11:28 +0800
Subject: [PATCH v5 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 17 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 34 +++-
src/backend/replication/logical/launcher.c | 7 +-
src/backend/replication/logical/worker.c | 10 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 184 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
15 files changed, 240 insertions(+), 90 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 964c819a02..039bd33d89 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8046,6 +8046,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c44b0e8108..f657e56368 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1644,7 +1644,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 8a3096e62b..b0a04ba90a 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -428,6 +428,23 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 9efc9159f2..849af83f1f 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -72,6 +72,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->passwordrequired = subform->subpasswordrequired;
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2e306ceff6..c38881d61a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1358,7 +1358,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)
+ subdetectupdatedeleted, 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 02ccc636b8..5b06698fda 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -594,7 +608,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -701,6 +716,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1196,7 +1213,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1356,6 +1373,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ ApplyLauncherWakeupAtCommit();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c31d4dcedf..8b1b02a331 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,6 +1190,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
{
can_advance_xmin = false;
@@ -1253,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated = advance_conflict_slot_xmin(xmin);
compute_slot_update_naptime(updated, &slot_update_wait_time);
@@ -1269,6 +1273,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c29b8f86c0..fb0c18709e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2745,7 +2745,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(localrel);
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -3098,7 +3099,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(partrel);
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -4007,6 +4009,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
Assert(remote_wal_pos);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1b47c388ce..078967e3a3 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4850,6 +4850,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4922,11 +4923,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4965,6 +4972,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5011,6 +5019,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5251,6 +5261,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9f907ed5ad..bd29448060 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -673,6 +673,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 363a66e718..2e275e7443 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6543,7 +6543,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)
{
@@ -6611,6 +6611,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 0aa14ec4a2..6f141df950 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 17d48b1685..52df143112 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | 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_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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | off | 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 | 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 | Detect update deleted | 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;
@@ -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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 | Detect update deleted | 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 | Detect update deleted | 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,50 @@ 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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = false);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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)
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | t | 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 007c9e7037..94c9aacd6f 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- now it works, but will report a warning due to disabled track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = false);
+
+\dRs+
+
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
Hi Hou-san, here are my review comments for patch v5-0001.
======
General
1.
Sometimes in the commit message and code comments the patch refers to
"transaction id" and other times to "transaction ID". The patch should
use the same wording everywhere.
======
Commit message.
2.
"While for concurrent remote transactions with earlier timestamps,..."
I think this means:
"But, for concurrent remote transactions with earlier timestamps than
the DELETE,..."
Maybe expressed this way is clearer.
~~~
3.
... the resolution would be to convert the update to an insert.
Change this to uppercase like done elsewhere:
"... the resolution would be to convert the UPDATE to an INSERT.
======
doc/src/sgml/protocol.sgml
4.
+ <varlistentry
id="protocol-replication-primary-wal-status-update">
+ <term>Primary WAL status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary WAL status update.
+ </para>
+ </listitem>
+ </varlistentry>
I felt it would be better if this is described as just a "Primary
status update" instead of a "Primary WAL status update". Doing this
makes it more flexible in case there is a future requirement to put
more status values in here which may not be strictly WAL related.
~~~
5.
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Standby WAL status request (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('W')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for the WAL status
on the primary.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
5a.
Ditto the previous comment #4. Perhaps you should just call this a
"Primary status request".
~
5b.
Also, The letter 'W' also seems chosen because of WAL. But it might be
more flexible if those identifiers are more generic.
e.g.
's' = the request for primary status update
'S' = the primary status update
======
src/backend/replication/logical/worker.c
6.
+ else if (c == 's')
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
Since there's no equivalent #define or enum value, IMO it is too hard
to know the intent of this code without already knowing the meaning of
the magic letter 's'. At least there could be a comment here to
explain that this is for handling an incoming "Primary status update"
message.
~~~
maybe_advance_nonremovable_xid:
7.
+ * 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-delete conflicts.
/update-delete/update_deleted/
~
8.
+ * applied and flushed locally. The process involves:
+ *
+ * DTR_REQUEST_WALSENDER_WAL_POS - Call GetOldestActiveTransactionId() to get
+ * the candidate xmin and send a message to request the remote WAL position
+ * from the walsender.
+ *
+ * DTR_WAIT_FOR_WALSENDER_WAL_POS - Wait for receiving the WAL position from
+ * the walsender.
+ *
+ * DTR_WAIT_FOR_LOCAL_FLUSH - Advance the non-removable transaction ID if the
+ * current flush location has reached or surpassed the received WAL position.
8a.
This part would be easier to read if those 3 phases were indented from
the rest of this function comment.
~
8b.
/Wait for receiving/Wait to receive/
~
9.
+ * Retaining the dead tuples for this period is sufficient for ensuring
+ * eventual consistency using last-update-wins strategy, which involves
+ * converting an UPDATE to an INSERT and applying it if remote transactions
The commit message referred to a "latest_timestamp_wins". I suppose
that is the same as what this function comment called
"last-update-wins". The patch should use consistent terminology.
It would be better if the commit message and (parts of) this function
comment were just cut/pasted to be identical. Currently, they seem to
be saying the same thing, but using slightly different wording.
~
10.
+ static TimestampTz xid_advance_attemp_time = 0;
+ static FullTransactionId candidate_xid;
typo in var name - "attemp"
~
11.
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+
+ /*
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_wal_pos. Proceeding to the next
+ * phase to check if we can immediately advance the transaction ID.
+ */
11a.
IMO this comment should be above the *phase assignment.
11b.
/Proceeding to the next phase to check.../Instead, proceed to the next
phase to check.../
~
12.
+ /*
+ * Advance the non-removable transaction id if the remote wal position
+ * has been received, and all transactions up to that position on the
+ * publisher have been applied and flushed locally.
+ */
Some minor re-wording would help clarify this comment.
SUGGESTION
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, now we can
advance the non-removable transaction id.
~
13.
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * wal position request. Proceeding to the next phase to determine if
+ * we can send the next request.
+ */
13a.
IMO this comment should be above the *phase assignment.
~
13b.
This comment should have the same wording here as in the previous
if-block (see #11b).
/Proceeding to the next phase to determine.../Instead, proceed to the
next phase to check.../
~
14.
+ FullTransactionId next_full_xix;
+ FullTransactionId full_xid;
You probably mean 'next_full_xid' (not xix)
~
15.
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
What are the implications of this early exit? If the update request is
not possible, then I guess the update status is never received, but
then I suppose that means none of this update_deleted logic is
possible. If that is correct, then will there be some documented
warning/caution about conflict-handling implications by disabling that
GUC?
======
src/backend/replication/walsender.c
16.
+/*
+ * Process the standby message requesting the latest WAL write position.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
Ideally, this function comment should be referring to this message we
are creating by the same name that it was called in the documentation.
For example something like:
"Process the request for a primary status update message."
======
Kind Regards,
Peter Smith.
Fujitsu Australia
Dear Hou,
Here is the V5 patch set which addressed above comments.
Thanks for updating the patch! While reviewing yours, I found a corner case that
a recently deleted tuple cannot be detected when index scan is chosen.
This can happen when indices are re-built during the replication.
Unfortunately, I don't have any solutions for it.
Found issue
========
When indices are built with the CONCURRENTLY option, a standard MVCC snapshot
is used to list up the tuples of the table, which means the new index ignores
recently deleted tuples.
This can cause the update_deleted to be wrongly detected as update_missing.
Assuming that we have a bidirectional cluster like case 1 [1]/messages/by-id/OS0PR01MB5716A78EE3D6F2F4754E1209946C2@OS0PR01MB5716.jpnprd01.prod.outlook.com (id is a primary key),
and REINDEX CONCURRENTLY happens after executing DELETE but before receiving the UPDATE.
A primary key of t will be re-built by the REINDEX command but the dead tuples by
DELETE will be missed because it is invisible from the transaction.
Then, the apply worker receives the UPDATE and recognizes the target tuple is removed.
It scans with snapshotany to find the deleted tuple via the index, but it fails.
This event is reported as update_missing.
Reproduce steps
===========
This can be reproduced with v5 patch set:
1. constructed a 2-way replication by running attached.
2. stopped an apply worker on node2
3. executed `UPDATE tab SET a = 2;` on node1.
4. executed `DELETE FROM tab;` on node 2
5. executed `REINDEX TABLE CONCURRENTLY tab;` on node2
6. resumed the stopped apply worker
7. the worker should detect update_deleted, but it detected update_missing
```
LOG: conflict detected on relation "public.tab": conflict=update_missing
DETAIL: Could not find the row to be updated.
Remote tuple (2); replica identity (a)=(1).
``
[1]: /messages/by-id/OS0PR01MB5716A78EE3D6F2F4754E1209946C2@OS0PR01MB5716.jpnprd01.prod.outlook.com
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Attachments:
Here is the V5 patch set which addressed above comments.
Here are a couple of comments on v5 patch-set -
1) In FindMostRecentlyDeletedTupleInfo(),
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ Assert(tuples_equal(scanslot, searchslot, eq));
+ update_recent_dead_tuple_info(scanslot, oldestXmin, delete_xid,
+ delete_time, delete_origin);
+ }
In my tests, I found that the above assert() triggers during
unidirectional replication of an update on a table. While doing the
replica identity index scan, it can only ensure to match the indexed
columns value, but the current Assert() assumes all the column values
should match, which seems wrong.
2) Since update_deleted requires both 'track_commit_timestamp' and the
'detect_update_deleted' to be enabled, should we raise an error in the
CREATE and ALTER subscription commands when track_commit_timestamp=OFF
but the user specifies detect_update_deleted=true?
Hello, Hayato!
Thanks for updating the patch! While reviewing yours, I found a corner
case that
a recently deleted tuple cannot be detected when index scan is chosen.
This can happen when indices are re-built during the replication.
Unfortunately, I don't have any solutions for it.
I just randomly saw your message, so, I could be wrong and out of the
context - so, sorry in advance.
But as far as I know, to solve this problem, we need to wait for slot.xmin
during the [0] (WaitForOlderSnapshots) while creating index concurrently.
[1]: https://github.com/postgres/postgres/blob/68dfecbef210dc000271553cfcb2342989d4ca0f/src/backend/commands/indexcmds.c#L1758-L1765
https://github.com/postgres/postgres/blob/68dfecbef210dc000271553cfcb2342989d4ca0f/src/backend/commands/indexcmds.c#L1758-L1765
Best regards,
Mikhail.
On Thursday, October 24, 2024 1:00 PM Peter Smith <smithpb2250@gmail.com> wrote:
Thanks Peter, Kuroda-san and Nisha for the comments.
Hi Hou-san, here are my review comments for patch v5-0001.
15. + /* + * Exit early if the user has disabled sending messages to the + * publisher. + */ + if (wal_receiver_status_interval <= 0) return;What are the implications of this early exit? If the update request is not possible,
then I guess the update status is never received, but then I suppose that means
none of this update_deleted logic is possible. If that is correct, then will there
be some documented warning/caution about conflict-handling implications by
disabling that GUC?
The detection of update_deleted is still possible in this case, but the dead tuple
cannot be cleaned up without communicating with the publisher. I have documented
this in the 0004 patch where the subscription option is added.
On Friday, October 25, 2024 2:17 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Here is the V5 patch set which addressed above comments.
Thanks for updating the patch! While reviewing yours, I found a corner case
that
a recently deleted tuple cannot be detected when index scan is chosen.
This can happen when indices are re-built during the replication.
Unfortunately, I don't have any solutions for it.
Thanks for reporting. I changed the patch to use table scan in all cases, since
It is used only in conflict cases, so it could be OK, but I will
think if there are better solutions for it based on Mikhail's reply[1]/messages/by-id/CANtu0ohzMB38tcf2=q1HVCdz+eEpBBauHhMoATz_=YFEjDd7FQ@mail.gmail.com..
On Friday, October 25, 2024 4:51 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
1) In FindMostRecentlyDeletedTupleInfo(),
+ /* Try to find the tuple */ + while (index_getnext_slot(scan, ForwardScanDirection, scanslot)) { + Assert(tuples_equal(scanslot, searchslot, eq)); + update_recent_dead_tuple_info(scanslot, oldestXmin, delete_xid, + delete_time, delete_origin); + }In my tests, I found that the above assert() triggers during unidirectional
replication of an update on a table. While doing the replica identity index scan,
it can only ensure to match the indexed columns value, but the current Assert()
assumes all the column values should match, which seems wrong.
Thanks for reporting. These codes have been removed in V6 patch set.
2) Since update_deleted requires both 'track_commit_timestamp' and the
'detect_update_deleted' to be enabled, should we raise an error in the CREATE
and ALTER subscription commands when track_commit_timestamp=OFF but
the user specifies detect_update_deleted=true?
Added.
Attach the V6 patch set which addressed all the comments so far.
[1]: /messages/by-id/CANtu0ohzMB38tcf2=q1HVCdz+eEpBBauHhMoATz_=YFEjDd7FQ@mail.gmail.com.
Best Regards,
Hou zj
Attachments:
v6-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v6-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From 4cf9828266155f569d0f784bae1affb5f46a667c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v6 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 153 ++++++++++++++++++
2 files changed, 154 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..ab239dfbd3
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,153 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.31.1
v6-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v6-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 63dea3a379f7677a431cee42ef2fd4a7ed049725 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v6 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 145 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 53 +++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 240 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 98a7ad0c27..c44b0e8108 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1637,6 +1637,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 331315f8d3..e2cd5ec07f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3456b821bc..2e306ceff6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..57aadddda6 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,134 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+
+ if (OidIsValid(localindexoid))
+ indexbitmap = build_index_column_bitmap(localindexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 d66d328174..bf16925687 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2688,6 +2688,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2704,15 +2707,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2723,7 +2722,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2742,19 +2741,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(localrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3076,7 +3084,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3086,17 +3094,26 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ Oid replica_index = GetRelationIdentityOrPK(partrel);
+
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ replica_index, &localxid,
+ &localts, &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3108,7 +3125,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3119,7 +3136,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index f7b50e0b5a..0a3353eb5c 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1978,7 +1978,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2006,11 +2006,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 1ec0d6f6b5..1e2a9c9362 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5597,9 +5597,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..bb04ef86c2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ Oid localindexoid,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2b47013f11..4dad6e4966 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2145,11 +2145,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v6-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v6-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From 4e63c774808a5c876944ca3cb08f824712eadaed Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@fujitsu.com>
Date: Mon, 28 Oct 2024 10:29:21 +0800
Subject: [PATCH v6 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 46 +++++-
src/backend/replication/logical/launcher.c | 7 +-
src/backend/replication/logical/worker.c | 10 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
15 files changed, 252 insertions(+), 90 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 964c819a02..039bd33d89 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8046,6 +8046,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c44b0e8108..f657e56368 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1644,7 +1644,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 8a3096e62b..bed47f766c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -428,6 +428,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2e306ceff6..c38881d61a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1358,7 +1358,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)
+ subdetectupdatedeleted, 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 33575614e7..e07456ebd5 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -563,7 +577,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -608,6 +623,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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1165,7 +1188,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,6 +1348,21 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
+ ApplyLauncherWakeupAtCommit();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index d77e38bf14..bc9a30007d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,6 +1190,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
{
can_advance_xmin = false;
@@ -1253,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated = advance_conflict_slot_xmin(xmin);
@@ -1270,6 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index bf16925687..d2710055db 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2745,7 +2745,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(localrel);
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -3098,7 +3099,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *newslot = localslot;
Oid replica_index = GetRelationIdentityOrPK(partrel);
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
replica_index, &localxid,
&localts, &localorigin) &&
localorigin != replorigin_session_origin)
@@ -4011,6 +4013,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
Assert(remote_wal_pos);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1b47c388ce..078967e3a3 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4850,6 +4850,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4922,11 +4923,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4965,6 +4972,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5011,6 +5019,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5251,6 +5261,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9f907ed5ad..bd29448060 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -673,6 +673,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 363a66e718..2e275e7443 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6543,7 +6543,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)
{
@@ -6611,6 +6611,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 17d48b1685..c3914ee0bf 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | 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_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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | off | 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 | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | off | 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 | 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 | Detect update deleted | 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;
@@ -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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | 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 | Detect update deleted | 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)
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 | off | 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 | Detect update deleted | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+-----------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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 - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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)
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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v6-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchapplication/octet-stream; name=v6-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchDownload
From ec592c496995a35e7b3fef29a91123999c43ce07 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v6 1/5] 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_delete
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 and send a message to request the remote WAL position from
the walsender.
2) Wait to receive the WAL position from the walsender.
3) Advance the non-removable transaction ID if the current flush location
has reached or surpassed the received WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 52 ++++++
src/backend/replication/logical/worker.c | 199 +++++++++++++++++++++-
src/backend/replication/walsender.c | 23 +++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 292 insertions(+), 1 deletion(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 057c46f3f5..a9d931109b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2440,6 +2440,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2584,6 +2619,23 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Primary status request (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..d66d328174 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,8 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +395,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3592,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3715,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3737,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3838,7 +3871,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;
@@ -3916,6 +3948,171 @@ 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:
+ *
+ * - DTR_REQUEST_WALSENDER_WAL_POS:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * - DTR_WAIT_FOR_WALSENDER_WAL_POS:
+ * Wait to receive the WAL position from the walsender.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * Advance the non-removable transaction ID if the current flush location has
+ * reached or surpassed the received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_wal_pos);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attempt_time);
+
+ /*
+ * Return if we have requested but not yet received the remote wal
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_wal_pos))
+ return;
+
+ /*
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_wal_pos. Instead, proceed to the
+ * next phase to check if we can immediately advance the transaction
+ * ID.
+ */
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_wal_pos) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *remote_wal_pos)
+ 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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * wal position request. Instead, proceed to the next phase to check
+ * if we can send the next request.
+ */
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ elog(DEBUG2, "sending wal position request message");
+
+ /* Send a wal position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "S", sizeof(uint8));
+
+ *remote_wal_pos = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 371eef3ddd..98451557bb 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,24 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ elog(DEBUG2, "sending wal write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, GetXLogWriteRecPtr());
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 171a7dd5d2..5d3faba798 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v6-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v6-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From 5231f2223152803f02c939905001629d2bf013dd Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v6 2/5] 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.
---
src/backend/replication/logical/launcher.c | 196 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 223 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..d77e38bf14 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1248,30 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1299,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 98451557bb..f464ec61b7 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
Hi Hou-San, here are a few trivial comments remaining for patch v6-0001.
======
General.
1.
There are multiple comments in this patch mentioning 'wal' which
probably should say 'WAL' (uppercase).
~~~
2.
There are multiple comments in this patch missing periods (.)
======
doc/src/sgml/protocol.sgml
3.
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
Currently, there are identifiers 's' for the "Primary status update"
message, and 'S' for the "Primary status request" message.
As mentioned in the previous review ([1]/messages/by-id/CAHut+Ps3sgXh2=rHDaqjU=p28CK5rCgCLJZgPByc6Tr7_P2imw@mail.gmail.com #5b) I preferred it to be the
other way around:
'S' = status from primary
's' = request status from primary
Of course, it doesn't make any difference, but "S" seems more
important than "s", so therefore "S" being the main msg and coming
from the *primary* seemed more natural to me.
~~~
4.
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Primary status request (F)</term>
Is it better to call this slightly differently to emphasise this is
only the request?
/Primary status request/Request primary status update/
======
src/backend/replication/logical/worker.c
5.
+ * 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
As mentioned in review [1]/messages/by-id/CAHut+Ps3sgXh2=rHDaqjU=p28CK5rCgCLJZgPByc6Tr7_P2imw@mail.gmail.com #9, this is still called "last-update-wins
strategy" here in the comment, but was called the "last update win
strategy" strategy in the commit message. Those terms should be the
same -- e.g. the 'last-update-wins' strategy.
======
[1]: /messages/by-id/CAHut+Ps3sgXh2=rHDaqjU=p28CK5rCgCLJZgPByc6Tr7_P2imw@mail.gmail.com
Kind Regards,
Peter Smith.
Fujitsu Australia
Dear Mikhail,
Thanks for giving comments!
But as far as I know, to solve this problem, we need to wait for slot.xmin during the [0]
(WaitForOlderSnapshots) while creating index concurrently.
WaitForOlderSnapshots() waits other transactions which can access older tuples
than the specified (=current) transaction, right? I think it does not solve our issue.
Assuming that same workloads [1]/messages/by-id/TYAPR01MB5692541820BCC365C69442FFF54F2@TYAPR01MB5692.jpnprd01.prod.outlook.com are executed, slot.xmin on node2 is arbitrary
older than noted SQL, and WaitForOlderSnapshots(slot.xmin) is added in
ReindexRelationConcurrently(). In this case, transaction older than slot.xmin
does not exist at step 5, so the REINDEX will finish immediately. Then, the worker
receives changes at step 7 so it is problematic if worker uses the reindexed index.
From another point of view... this approach must fix REINDEX code, but we should
not modify other component of codes as much as possible. This feature is related
with the replication so that changes should be closed within the replication subdir.
[1]: /messages/by-id/TYAPR01MB5692541820BCC365C69442FFF54F2@TYAPR01MB5692.jpnprd01.prod.outlook.com
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Hello Hayato,
WaitForOlderSnapshots() waits other transactions which can access older
tuples
than the specified (=current) transaction, right? I think it does not
solve our issue.
Oh, I actually described the idea a bit incorrectly. The goal isn’t simply
to call WaitForOlderSnapshots(slot.xmin);
rather, it’s to ensure that we wait for slot.xmin in the same way we wait
for regular snapshots (xmin).
The reason WaitForOlderSnapshots is used in ReindexConcurrently and
DefineIndex is to guarantee that any transaction
needing to view rows not included in the index has completed before the
index is marked as valid.
The same logic should apply here — we need to wait for the xmin of slot
used in conflict detection as well.
From another point of view... this approach must fix REINDEX code, but we
should
not modify other component of codes as much as possible. This feature is
related
with the replication so that changes should be closed within the
replication subdir.
One possible solution here would be to register a snapshot with slot.xmin
for the worker backend.
This way, WaitForOlderSnapshots will account for it.
By the way, WaitForOlderSnapshots is also used in partitioning and other
areas for similar reasons,
so these might be good places to check for any related issues.
Best regards,
Mikhail,
Dear Mikhail,
Thanks for describing more detail!
Oh, I actually described the idea a bit incorrectly. The goal isn’t simply to call WaitForOlderSnapshots(slot.xmin);
rather, it’s to ensure that we wait for slot.xmin in the same way we wait for regular snapshots (xmin).
...
One possible solution here would be to register a snapshot with slot.xmin for the worker backend.
This way, WaitForOlderSnapshots will account for it.
Note that apply workers can stop due to some reasons (e.g., disabling subscriptions,
error out, deadlock...). In this case, the snapshot cannot eb registered by the
worker and index can be re-built during the period.
If we do not assume the existence of workers, we must directly somehow check slot.xmin
and wait until it is advanced until the REINDEXing transaction. I still think it
is risky and another topic.
Anyway, this topic introduces huge complexity and is not mandatory for update_deleted
detection. We can work on it in later versions based on the needs.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Dear Hou,
Thanks for updating the patch! Here are my comments.
01. CreateSubscription
```
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
```
I don't think this guard is sufficient. I found two cases:
* Creates a subscription with detect_update_deleted = false and track_commit_timestamp = true,
then alters detect_update_deleted to true.
* Creates a subscription with detect_update_deleted = true and track_commit_timestamp = true,
then update track_commit_timestamp to true and restart the instance.
Based on that, how about detecting the inconsistency on the apply worker? It check
the parameters and do error out when it starts or re-reads a catalog. If we want
to detect in SQL commands, this can do in parse_subscription_options().
02. AlterSubscription()
```
+ ApplyLauncherWakeupAtCommit();
```
The reason why launcher should wake up is different from other parts. Can we add comments
that it is needed to track/untrack the xmin?
03. build_index_column_bitmap()
```
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
```
I feel we can assert the ii_IndexAttrNumbers is valid, because the passed index is a replica identity key.
04. LogicalRepApplyLoop()
Can we move the definition of "phase" to the maybe_advance_nonremovable_xid() and
make it static? The variable is used only by the function.
05. LogicalRepApplyLoop()
```
+ UpdateWorkerStats(last_received, timestamp, false);
```
The statistics seems not correct. last_received is not sent at "timestamp", it had
already been sent earlier. Do we really have to update here?
06. ErrorOnReservedSlotName()
I feel we should document that the slot name 'pg_conflict_detection' cannot be specified
unconditionally.
07. General
update_deleted can happen without DELETE commands. Should we rename the conflict
reason, like 'update_target_modified'?
E.g., there is a 2-way replication system and below transactions happen:
Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts = 10.00
T2: UPDATE t SET id = 2 WHERE id = 1; ts = 10.02
Node B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts = 10.01
Then, T3 comes to Node A after executing T2. T3 tries to find id = 1 but find a
dead tuple instead. In this case, 'update_delete' happens without the delete.
08. Others
Also, here is an analysis related with the truncation of commit timestamp. I worried the
case that commit timestamp might be removed so that the detection would not go well.
But it seemed that entries can be removed when it behinds GetOldestNonRemovableTransactionId(NULL),
i.e., horizons.shared_oldest_nonremovable. The value is affected by the replication
slots so that interesting commit_ts entries for us are not removed.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Hello Hayato!
Note that apply workers can stop due to some reasons (e.g., disabling
subscriptions,
error out, deadlock...). In this case, the snapshot cannot eb registered
by the
worker and index can be re-built during the period.
However, the xmin of a slot affects replication_slot_xmin in
ProcArrayStruct, so it might
be straightforward to wait for it during concurrent index builds. We could
consider adding
a separate conflict_resolution_replication_slot_xmin to wait only for that.
Anyway, this topic introduces huge complexity and is not mandatory for
update_deleted
detection. We can work on it in later versions based on the needs.
From my perspective, this is critical for databases. REINDEX CONCURRENTLY
is typically run
in production databases on regular basic, so any master-master system
should be unaffected by it.
Best regards,
Mikhail.
Dear Mikhail,
Thanks for the reply!
Anyway, this topic introduces huge complexity and is not mandatory for update_deleted
detection. We can work on it in later versions based on the needs.From my perspective, this is critical for databases. REINDEX CONCURRENTLY is typically run
in production databases on regular basic, so any master-master system should be unaffected by it.
I think you do not understand what I said correctly. The main point here is that
the index scan is not needed to detect the update_deleted. In the first version
workers can do the normal sequential scan instead. This workaround definitely does
not affect REINDEX CONCURRENTLY.
After the patch being good shape or pushed, we can support using the index to find
the dead tuple, at that time we can consider how we ensure the index contains the entry
for dead tuples.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Monday, October 28, 2024 1:40 PM Peter Smith <smithpb2250@gmail.com> wrote:
Hi Hou-San, here are a few trivial comments remaining for patch v6-0001.
Thanks for the comments!
======
doc/src/sgml/protocol.sgml3. + <term>Primary status update (B)</term> + <listitem> + <variablelist> + <varlistentry> + <term>Byte1('s')</term>Currently, there are identifiers 's' for the "Primary status update"
message, and 'S' for the "Primary status request" message.As mentioned in the previous review ([1] #5b) I preferred it to be the other way
around:
'S' = status from primary
's' = request status from primaryOf course, it doesn't make any difference, but "S" seems more important than
"s", so therefore "S" being the main msg and coming from the *primary*
seemed more natural to me.
I am not sure if one message is more important than another, so I prefer to
keep the current style. Since this is a minor issue, we can easily revise it in
future version patches if we receive additional feedback.
Other comments look good to me and will address in V7 patch set.
Best Regards,
Hou zj
On Wednesday, October 30, 2024 4:34 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Thanks for updating the patch! Here are my comments.
Thanks for giving comments !
01. CreateSubscription ``` + if (opts.detectupdatedeleted && !track_commit_timestamp) + ereport(ERROR, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled", + "track_commit_timestamp")); ```I don't think this guard is sufficient. I found two cases:
* Creates a subscription with detect_update_deleted = false and
track_commit_timestamp = false,
then alters detect_update_deleted to true.
I think this is not allowed, as we already have a check in AlterSubscription().
* Creates a subscription with detect_update_deleted = true and
track_commit_timestamp = true,
then update track_commit_timestamp to true and restart the instance.Based on that, how about detecting the inconsistency on the apply worker? It
check
the parameters and do error out when it starts or re-reads a catalog.
I agree to add the check in apply worker. But I would prefer to keep the check in
DDL as well, as I think It's better to prevent the bad configuration in the
beginning.
If we want
to detect in SQL commands, this can do in parse_subscription_options().
I personally feel parse_subscription_options() is not an appropriate place for this kind of
configuration error. So, I prefer to keep the check in CreateSubscription and
AlterSubscription.
03. build_index_column_bitmap() ``` + for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++) + { + int keycol = indexinfo->ii_IndexAttrNumbers[i]; + + index_bitmap = bms_add_member(index_bitmap, keycol); + } ```I feel we can assert the ii_IndexAttrNumbers is valid, because the passed index
is a replica identity key.
I changed to directly get the replica identity index in the
FindMostRecentlyDeletedTupleInfo instead of passing by parameter. After that, I
think the assert is not necessary as the code is straightforward.
05. LogicalRepApplyLoop()
```
+ UpdateWorkerStats(last_received, timestamp,
false);
```The statistics seems not correct. last_received is not sent at "timestamp", it
had
already been sent earlier. Do we really have to update here?
The timestamp updated here is used in the view
pg_stat_subscription.last_msg_send_time, which is a general timestamp that is
not directly related to the LSN received. See the desc in doc:
"""
last_msg_send_time
Send time of last message received from origin WAL sender; NULL for
parallel apply workers
"""
So, I think it's correct to update the timestamp here while keep last received lsn
unchanged.
06. ErrorOnReservedSlotName()
I feel we should document that the slot name 'pg_conflict_detection' cannot be
specified
unconditionally.
I am a bit not sure if it's necessary, as the error message is clear that this
is a reserved name. The doc of the detect_update_delete also mentioned the name
of this slot.
When checking other options, we have reserved 'replication origin' name as well
"NONE,ANY" which are not explicitly mentioned in the doc. So, it seems fine.
07. General
update_deleted can happen without DELETE commands. Should we rename
the conflict
reason, like 'update_target_modified'?
I am not sure if 'update_target_modified' is appropriate, as it's too similar to
the existing 'update_origin_change' conflict ("Updating a row that was
previously modified by another origin.").
Since the UPDATE can also be considered as DELETE + INSERT, so the current name
looks fine to me.
Attach the V7 patch set which address all other comments not mentioned.
Best Regards,
Hou zj
Attachments:
v7-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v7-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From e3b533acf7b954f79f26ff1d0d0f45c913fd234b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v7 2/5] 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.
---
src/backend/replication/logical/launcher.c | 196 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 223 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..d77e38bf14 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1248,30 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1299,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index ef62a1d066..f0b4390385 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.31.1
v7-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchapplication/octet-stream; name=v7-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchDownload
From 507fc3ddde19e72af5146c539088f70d71833cff Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v7 1/5] 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_delete
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 and send a message to request the remote WAL position from
the walsender.
2) Wait to receive the WAL position from the walsender.
3) Advance the non-removable transaction ID if the current flush location
has reached or surpassed the received WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 52 ++++++
src/backend/replication/logical/worker.c | 197 +++++++++++++++++++++-
src/backend/replication/walsender.c | 23 +++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 290 insertions(+), 1 deletion(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 057c46f3f5..3f8f885696 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2440,6 +2440,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2584,6 +2619,23 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..2bfa13d34c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,8 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +395,7 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3591,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3713,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ maybe_advance_nonremovable_xid(&remote_lsn);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3735,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3838,7 +3869,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;
@@ -3916,6 +3946,171 @@ 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:
+ *
+ * - DTR_REQUEST_WALSENDER_WAL_POS:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * - DTR_WAIT_FOR_WALSENDER_WAL_POS:
+ * Wait to receive the WAL position from the walsender.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * Advance the non-removable transaction ID if the current flush location has
+ * reached or surpassed the received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_wal_pos' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+ static DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
+
+ Assert(remote_wal_pos);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attempt_time);
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_wal_pos))
+ return;
+
+ /*
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_wal_pos. Instead, proceed to the
+ * next phase to check if we can immediately advance the transaction
+ * ID.
+ */
+ phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+
+ if (phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_wal_pos) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *remote_wal_pos)
+ 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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * WAL position request. Instead, proceed to the next phase to check
+ * if we can send the next request.
+ */
+ phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ elog(DEBUG2, "sending WAL position request message");
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn, "S", sizeof(uint8));
+
+ *remote_wal_pos = InvalidXLogRecPtr;
+ phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 371eef3ddd..ef62a1d066 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,24 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ elog(DEBUG2, "sending WAL write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, GetXLogWriteRecPtr());
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 171a7dd5d2..5d3faba798 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.31.1
v7-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v7-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 6b3ef64dd540e2a3b986e49eefa9159cc5a0a5b2 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v7 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 98a7ad0c27..c44b0e8108 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1637,6 +1637,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 331315f8d3..e2cd5ec07f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3456b821bc..2e306ceff6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..9e1f15bb6a 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 2bfa13d34c..0e8d1c9c2b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2687,6 +2687,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2703,15 +2706,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2722,7 +2721,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2741,19 +2740,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localts,
+ &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3075,7 +3082,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3085,17 +3092,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localts,
+ &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3107,7 +3122,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3118,7 +3133,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index f7b50e0b5a..0a3353eb5c 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1978,7 +1978,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2006,11 +2006,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 1ec0d6f6b5..1e2a9c9362 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5597,9 +5597,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..07b2c10892 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 2b47013f11..4dad6e4966 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2145,11 +2145,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.31.1
v7-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v7-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From 1b4ea515d9d2bd6b299f02af9adbb7d6f937d172 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 5 Nov 2024 09:55:02 +0800
Subject: [PATCH v7 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 52 +++++-
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 7 +-
src/backend/replication/logical/worker.c | 10 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 265 insertions(+), 94 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 964c819a02..039bd33d89 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8046,6 +8046,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c44b0e8108..f657e56368 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1644,7 +1644,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 2e306ceff6..c38881d61a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1358,7 +1358,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)
+ subdetectupdatedeleted, 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 03e97730e7..25d2c4ce55 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -563,7 +577,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -608,6 +623,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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1165,7 +1188,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,6 +1348,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 9e1f15bb6a..b8a83790cb 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_time = 0;
*delete_origin = InvalidRepOriginId;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index d77e38bf14..bc9a30007d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,6 +1190,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
{
can_advance_xmin = false;
@@ -1253,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated = advance_conflict_slot_xmin(xmin);
@@ -1270,6 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0e8d1c9c2b..491dca6064 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2743,7 +2743,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localts,
&localorigin) &&
localorigin != replorigin_session_origin)
@@ -3095,7 +3096,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localts,
&localorigin) &&
localorigin != replorigin_session_origin)
@@ -4007,6 +4009,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_wal_pos)
Assert(remote_wal_pos);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index d8c6330732..27449bdca9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4850,6 +4850,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4922,11 +4923,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4965,6 +4972,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5011,6 +5019,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5253,6 +5263,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9f907ed5ad..bd29448060 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -673,6 +673,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 363a66e718..2e275e7443 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6543,7 +6543,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)
{
@@ -6611,6 +6611,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.31.1
v7-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v7-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From 8a2105ecca75a325dbe1f283168af130afaab729 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v7 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 153 ++++++++++++++++++
2 files changed, 154 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..1470ed87d8
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,153 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.31.1
On Friday, October 18, 2024 5:45 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Oct 15, 2024 at 5:03 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Mon, Oct 14, 2024 at 9:09 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:We thought of few options to fix this:
1) Add a Time-Based Subscription Option:
We could add a new time-based option for subscriptions, such as
retain_dead_tuples = '5s'. In the logical launcher, after obtaining
the candidate XID, the launcher will wait for the specified time
before advancing the slot.xmin. This ensures that deleted tuples are
retained for at least the duration defined by this new option.This approach is designed to simulate the functionality of the GUC
(vacuum_committs_age), but with a simpler implementation that does
not impact vacuum performance. We can maintain both this time-based
method and the current automatic method. If a user does not specify
the time-based option, we will continue using the existing approach
to retain dead tuples until all concurrent transactions from the remote nodehave been applied.
2) Modification to the Logical Walsender
On the logical walsender, which is as a physical standby, we can
build an additional connection to the physical primary to obtain the
latest WAL position. This position will then be sent as feedback to
the logical subscriber.A potential concern is that this requires the walsender to use the
walreceiver API, which may seem a bit unnatural. And, it starts an
additional walsender process on the primary, as the logical
walsender on the physical standby will need to communicate with thiswalsender to fetch the WAL position.
This idea is worth considering, but I think it may not be a good
approach if the physical standby is cascading. We need to restrict the
update_delete conflict detection, if the standby is cascading, right?The other approach is that we send current_timestamp from the
subscriber and somehow check if the physical standby has applied
commit_lsn up to that commit_ts, if so, it can send that WAL position
to the subscriber, otherwise, wait for it to be applied. If we do this
then we don't need to add a restriction for cascaded physical standby.
I think the subscriber anyway needs to wait for such an LSN to be
applied on standby before advancing the xmin even if we get it from
the primary. This is because the subscriber can only be able to apply
and flush the WAL once it is applied on the standby. Am, I missing
something?This approach has a disadvantage that we are relying on clocks to be
synced on both nodes which we anyway need for conflict resolution as
discussed in the thread [1]. We also need to consider the Commit
Timestamp and LSN inversion issue as discussed in another thread [2]
if we want to pursue this approach because we may miss an LSN that has
a prior timestamp.
For the "publisher is also a standby" issue, I have modified the V8 patch to
report a warning in this case. As I personally feel this is not the main use case
for conflict detection, we can revisit it later after pushing the main patches
receiving some user feedback.
The problem due to Commit Timestamp and LSN inversion is that the standby
may not consider the WAL LSN from an earlier timestamp, which could lead to
the removal of required dead rows on the subscriber.The other problem pointed out by Hou-San offlist due to Commit Timestamp
and LSN inversion is that we could miss sending the WAL LSN that the
subscriber requires to retain dead rows for update_delete conflict. For example,
consider the following case 2 node, bidirectional setup:Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AMNode B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AMSay subscription is created with retain_dead_tuples = true/false
After executing T2, the apply worker on Node A will check the latest wal flush
location on Node B. Till that time, the T3 should have finished, so the xmin will
be advanced only after applying the WALs that is later than T3. So, the dead
tuple will not be removed before applying the T3, which means the
update_delete can be detected.As there is a gap between when we acquire the commit_timestamp and the
commit LSN, it is possible that T3 would have not yet flushed it's WAL even
though it is committed earlier than T2. If this happens then we won't be able to
detect update_deleted conflict reliably.Now, the one simpler idea is to acquire the commit timestamp and reserve WAL
(LSN) under the same spinlock in
ReserveXLogInsertLocation() but that could be costly as discussed in the
thread [1]. The other more localized solution is to acquire a timestamp after
reserving the commit WAL LSN outside the lock which will solve this particular
problem.
Since the discussion of the WAL/LSN inversion issue is ongoing, I also thought
about another approach that can fix the issue independently. This idea is to
delay the non-removable xid advancement until all the remote concurrent
transactions that may have been assigned earlier timestamp have been committed.
The implementation is:
On the walsender, after receiving a request, it can send the oldest xid and
next xid along with the
In response, the apply worker can safely advance the non-removable XID if
oldest_committing_xid == nextXid, indicating that there is no race conditions.
Alternatively, if oldest_committing_xid != nextXid, the apply worker might send
a second request after some interval. If the subsequently obtained
oldest_committing_xid surpasses or equal to the initial nextXid, it indicates
that all previously risky transactions have committed, therefore the the
non-removable transaction ID can be advanced.
Attach the V8 patch set. Note that I put the new approach for above race
condition in a temp patch " v8-0001_2-Maintain-xxx.patch.txt", because the
approach may or may not be accepted based on the discussion in WAL/LSN
inversion thread.
[1] -
/messages/by-id/CAJpy0uBxEJnabEp3JS=n9X19V
x2ZK3k5AR7N0h-cSMtOwYV3fA%40mail.gmail.com
Best Regards,
Hou zj
Attachments:
v8-0001_2-Maintain-the-oldest-non-removeable-tranasction-I.patch.txttext/plain; name=v8-0001_2-Maintain-the-oldest-non-removeable-tranasction-I.patch.txtDownload
From b676827e76a21eea6a9b67d9b65aa673dd85705d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v88] 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_delete
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 running
transactions.
3) Wait for the status from the walsender. After receiving the first status
after acquiring a new candidate transaction ID, do not proceed if there are
ongoing concurrent remote transactions. 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 89 +++++++
src/backend/replication/logical/worker.c | 307 +++++++++++++++++++++-
src/backend/replication/walsender.c | 57 ++++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 470 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..3d0cf32356 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,68 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest running transaction ID on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2647,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..60d13c6e02 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,20 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_GET_CANDIDATE_XID,
+ DTR_REQUEST_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +355,10 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +398,11 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ TransactionId remote_oldestxid,
+ TransactionId remote_nextxid,
+ uint32 remote_epoch,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3598,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn;
+ TransactionId remote_oldestxid;
+ TransactionId remote_nextxid;
+ uint32 remote_epoch;
+ DeadTupleRetainPhase phase = DTR_GET_CANDIDATE_XID;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3724,50 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ remote_oldestxid = pq_getmsgint(&s, 4);
+ remote_nextxid = pq_getmsgint(&s, 4);
+ remote_epoch = pq_getmsgint(&s, 4);
+ timestamp = pq_getmsgint64(&s);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the request phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ maybe_advance_nonremovable_xid(&remote_lsn,
+ remote_oldestxid,
+ remote_nextxid,
+ remote_epoch,
+ &phase);
+
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3780,9 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, remote_oldestxid,
+ remote_nextxid, remote_epoch, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3910,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3991,234 @@ 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:
+ *
+ * - DTR_GET_CANDIDATE_XID:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid.
+ *
+ * - DTR_REQUEST_PUBLISHER_STATUS:
+ * Send a message to the walsender requesting the publisher status, which
+ * includes the latest WAL write position and information about running
+ * transactions.
+ *
+ * - DTR_WAIT_FOR_PUBLISHER_STATUS:
+ * Wait for the status from the walsender. After receiving the first status
+ * after acquiring a new candidate transaction ID, do not proceed if there
+ * are ongoing concurrent remote transactions. 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
+ * (DTR_REQUEST_PUBLISHER_STATUS) until all these transactions have
+ * completed.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * Advance the non-removable transaction ID if the current flush location has
+ * reached or surpassed the last received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_lsn' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ TransactionId remote_oldestxid,
+ TransactionId remote_nextxid,
+ uint32 remote_epoch,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+ static FullTransactionId next_phase_at;
+
+ TimestampTz now = 0;
+
+ Assert(remote_lsn);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_PUBLISHER_STATUS)
+ {
+ FullTransactionId remote_full_xid;
+
+ Assert(xid_advance_attempt_time);
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(next_phase_at))
+ next_phase_at = FullTransactionIdFromEpochAndXid(remote_epoch,
+ remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (remote_oldestxid > remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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.
+ *
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_lsn, or enough time might have
+ * passed. Instead, proceed to the next phase to check if we can
+ * immediately advance the transaction ID or send one more request.
+ */
+ if (FullTransactionIdPrecedesOrEquals(next_phase_at, remote_full_xid))
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ else
+ *phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_lsn) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * transaction ID advancement. Instead, proceed to the next phase to
+ * check if we can get the next candidate transaction ID.
+ */
+ *phase = DTR_GET_CANDIDATE_XID;
+ }
+
+ Assert(*phase == DTR_GET_CANDIDATE_XID ||
+ *phase == DTR_REQUEST_PUBLISHER_STATUS);
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+
+ if (*phase == DTR_GET_CANDIDATE_XID)
+ {
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+ next_phase_at = InvalidFullTransactionId;
+
+ /*
+ * Do not return here to ensure an immediate request message is sent
+ * in the next phase.
+ */
+ *phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ if (*phase == DTR_REQUEST_PUBLISHER_STATUS)
+ {
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, now);
+
+ elog(DEBUG2, "sending publisher status request message");
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ *remote_lsn = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_PUBLISHER_STATUS;
+ }
+}
+
/*
* 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 371eef3ddd..63eed24c5a 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,57 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestRunningXid = InvalidTransactionId;
+ FullTransactionId nextFullXid = InvalidFullTransactionId;
+ WalSnd *walsnd = MyWalSnd;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * Information about running transactions and the WAL write position is
+ * only available on a non-standby server.
+ *
+ * XXX: We could consider forwarding the request to the primary server if
+ * the current server is a standby.
+ */
+ if (!RecoveryInProgress())
+ {
+ oldestRunningXid = GetOldestActiveTransactionId();
+ 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, oldestRunningXid);
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 1847bbfa95..d24cc956f0 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v8-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchapplication/octet-stream; name=v8-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchDownload
From 123f41efebf3a1f807024cb9224fa202e469b253 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v8 1/5] 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_delete
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 and send a message to request the remote WAL position from
the walsender.
2) Wait to receive the WAL position from the walsender.
3) Advance the non-removable transaction ID if the current flush location
has reached or surpassed the received WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 62 ++++++
src/backend/replication/logical/worker.c | 242 +++++++++++++++++++++-
src/backend/replication/walsender.c | 47 +++++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 368 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..023b690a43 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2620,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..f7e483b3ca 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,19 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +354,10 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +397,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3594,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3717,42 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the initial phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3765,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3894,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3975,185 @@ 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:
+ *
+ * - DTR_REQUEST_WALSENDER_WAL_POS:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * - DTR_WAIT_FOR_WALSENDER_WAL_POS:
+ * Wait to receive the WAL position from the walsender.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * Advance the non-removable transaction ID if the current flush location has
+ * reached or surpassed the received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_lsn' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_lsn);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attempt_time);
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_lsn))
+ return;
+
+ /*
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_lsn. Instead, proceed to the
+ * next phase to check if we can immediately advance the transaction
+ * ID.
+ */
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_lsn) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * WAL position request. Instead, proceed to the next phase to check
+ * if we can send the next request.
+ */
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ elog(DEBUG2, "sending WAL position request message");
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, now); /* sendTime */
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ *remote_lsn = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_WALSENDER_WAL_POS;
+ }
+}
+
/*
* 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 371eef3ddd..19d3b23f42 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,48 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ WalSnd *walsnd = MyWalSnd;
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * The latest WAL write position is only available on a non-standby
+ * server.
+ *
+ * XXX: We could consider forwarding the request to the primary server if
+ * the current server is a standby.
+ */
+ if (!RecoveryInProgress())
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending WAL write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_delete conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 1847bbfa95..d24cc956f0 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v8-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v8-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From 60906577dc5cd0fb12729fdc40d317fb3d11fc3e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v8 2/5] 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.
---
src/backend/replication/logical/launcher.c | 196 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 223 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..d77e38bf14 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1248,30 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1299,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 19d3b23f42..576bfbd98d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v8-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v8-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 17406adf2c4d0eaf37ecef10ed25e5e3798b3f58 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v8 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index b7e340824c..1b63b58271 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1639,6 +1639,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..9e1f15bb6a 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 f7e483b3ca..80ceb821d6 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2690,6 +2690,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2706,15 +2709,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2725,7 +2724,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2744,19 +2743,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localts,
+ &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3078,7 +3085,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3088,17 +3095,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localts,
+ &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3110,7 +3125,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3121,7 +3136,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..07b2c10892 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v8-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v8-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From fb143e5923bc2ccad4be12e1befbde3509171c0b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 5 Nov 2024 09:55:02 +0800
Subject: [PATCH v8 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 52 +++++-
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 7 +-
src/backend/replication/logical/worker.c | 10 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 265 insertions(+), 94 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c180ed7abb..f00155ae65 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8042,6 +8042,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 1b63b58271..416fb0f78d 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1646,7 +1646,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..25d2c4ce55 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -563,7 +577,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -608,6 +623,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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1165,7 +1188,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,6 +1348,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 9e1f15bb6a..b8a83790cb 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_time = 0;
*delete_origin = InvalidRepOriginId;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index d77e38bf14..bc9a30007d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,6 +1190,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
{
can_advance_xmin = false;
@@ -1253,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated = advance_conflict_slot_xmin(xmin);
@@ -1270,6 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 80ceb821d6..439f316602 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2746,7 +2746,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localts,
&localorigin) &&
localorigin != replorigin_session_origin)
@@ -3098,7 +3099,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localts,
&localorigin) &&
localorigin != replorigin_session_origin)
@@ -4036,6 +4038,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
Assert(remote_lsn);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index a8c141b689..d56e81cb0a 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4861,6 +4861,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4933,11 +4934,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5022,6 +5030,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5264,6 +5274,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 5bfebad64d..fc2288d114 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6602,7 +6602,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)
{
@@ -6670,6 +6670,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v8-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v8-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From f692e4ae2af24245a9c2396ecd4136fab2897955 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v8 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 153 ++++++++++++++++++
2 files changed, 154 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..1470ed87d8
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,153 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'feedback_slots' parameter and its interaction
+# with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
On Tue, Nov 12, 2024 at 2:19 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, October 18, 2024 5:45 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Oct 15, 2024 at 5:03 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Mon, Oct 14, 2024 at 9:09 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:We thought of few options to fix this:
1) Add a Time-Based Subscription Option:
We could add a new time-based option for subscriptions, such as
retain_dead_tuples = '5s'. In the logical launcher, after obtaining
the candidate XID, the launcher will wait for the specified time
before advancing the slot.xmin. This ensures that deleted tuples are
retained for at least the duration defined by this new option.This approach is designed to simulate the functionality of the GUC
(vacuum_committs_age), but with a simpler implementation that does
not impact vacuum performance. We can maintain both this time-based
method and the current automatic method. If a user does not specify
the time-based option, we will continue using the existing approach
to retain dead tuples until all concurrent transactions from the remote nodehave been applied.
2) Modification to the Logical Walsender
On the logical walsender, which is as a physical standby, we can
build an additional connection to the physical primary to obtain the
latest WAL position. This position will then be sent as feedback to
the logical subscriber.A potential concern is that this requires the walsender to use the
walreceiver API, which may seem a bit unnatural. And, it starts an
additional walsender process on the primary, as the logical
walsender on the physical standby will need to communicate with thiswalsender to fetch the WAL position.
This idea is worth considering, but I think it may not be a good
approach if the physical standby is cascading. We need to restrict the
update_delete conflict detection, if the standby is cascading, right?The other approach is that we send current_timestamp from the
subscriber and somehow check if the physical standby has applied
commit_lsn up to that commit_ts, if so, it can send that WAL position
to the subscriber, otherwise, wait for it to be applied. If we do this
then we don't need to add a restriction for cascaded physical standby.
I think the subscriber anyway needs to wait for such an LSN to be
applied on standby before advancing the xmin even if we get it from
the primary. This is because the subscriber can only be able to apply
and flush the WAL once it is applied on the standby. Am, I missing
something?This approach has a disadvantage that we are relying on clocks to be
synced on both nodes which we anyway need for conflict resolution as
discussed in the thread [1]. We also need to consider the Commit
Timestamp and LSN inversion issue as discussed in another thread [2]
if we want to pursue this approach because we may miss an LSN that has
a prior timestamp.For the "publisher is also a standby" issue, I have modified the V8 patch to
report a warning in this case. As I personally feel this is not the main use case
for conflict detection, we can revisit it later after pushing the main patches
receiving some user feedback.The problem due to Commit Timestamp and LSN inversion is that the standby
may not consider the WAL LSN from an earlier timestamp, which could lead to
the removal of required dead rows on the subscriber.The other problem pointed out by Hou-San offlist due to Commit Timestamp
and LSN inversion is that we could miss sending the WAL LSN that the
subscriber requires to retain dead rows for update_delete conflict. For example,
consider the following case 2 node, bidirectional setup:Node A:
T1: INSERT INTO t (id, value) VALUES (1,1); ts=10.00 AM
T2: DELETE FROM t WHERE id = 1; ts=10.02 AMNode B:
T3: UPDATE t SET value = 2 WHERE id = 1; ts=10.01 AMSay subscription is created with retain_dead_tuples = true/false
After executing T2, the apply worker on Node A will check the latest wal flush
location on Node B. Till that time, the T3 should have finished, so the xmin will
be advanced only after applying the WALs that is later than T3. So, the dead
tuple will not be removed before applying the T3, which means the
update_delete can be detected.As there is a gap between when we acquire the commit_timestamp and the
commit LSN, it is possible that T3 would have not yet flushed it's WAL even
though it is committed earlier than T2. If this happens then we won't be able to
detect update_deleted conflict reliably.Now, the one simpler idea is to acquire the commit timestamp and reserve WAL
(LSN) under the same spinlock in
ReserveXLogInsertLocation() but that could be costly as discussed in the
thread [1]. The other more localized solution is to acquire a timestamp after
reserving the commit WAL LSN outside the lock which will solve this particular
problem.Since the discussion of the WAL/LSN inversion issue is ongoing, I also thought
about another approach that can fix the issue independently. This idea is to
delay the non-removable xid advancement until all the remote concurrent
transactions that may have been assigned earlier timestamp have been committed.The implementation is:
On the walsender, after receiving a request, it can send the oldest xid and
next xid along with theIn response, the apply worker can safely advance the non-removable XID if
oldest_committing_xid == nextXid, indicating that there is no race conditions.Alternatively, if oldest_committing_xid != nextXid, the apply worker might send
a second request after some interval. If the subsequently obtained
oldest_committing_xid surpasses or equal to the initial nextXid, it indicates
that all previously risky transactions have committed, therefore the the
non-removable transaction ID can be advanced.Attach the V8 patch set. Note that I put the new approach for above race
condition in a temp patch " v8-0001_2-Maintain-xxx.patch.txt", because the
approach may or may not be accepted based on the discussion in WAL/LSN
inversion thread.
I've started to review these patch series. I've reviewed only 0001
patch for now but let me share some comments:
---
+ if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS)
+ {
+ Assert(xid_advance_attempt_time);
What is this assertion for? If we want to check here that we have sent
a request message for the publisher, I think it's clearer if we have
"Assert(xid_advance_attempt_time > 0)". I'm not sure we really need
this assertion though since it's never false once we set
xid_advance_attempt_time.
---
+ /*
+ * Do not return here because the apply worker might
have already
+ * applied all changes up to remote_lsn. Instead,
proceed to the
+ * next phase to check if we can immediately advance
the transaction
+ * ID.
+ */
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
If we always proceed to the next phase, is this phase really
necessary? IIUC even if we jump to DTR_WAIT_FOR_LOCAL_FLUSH phase
after DTR_REQUEST_WALSENDER_WAL_POS and have a check if we received
the remote WAL position in DTR_WAIT_FOR_LOCAL_FLUSH phase, it would
work fine.
---
+ /*
+ * 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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
How about adding a debug log message showing new
oldest_nonremovable_xid and related LSN for making the
debug/investigation easier? For example,
elog(LOG, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(*remote_lsn),
XidFromFullTransactionId(candidate_xid));
---
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
In send_feedback(), we send a feedback message if the publisher
requests, even if wal_receiver_status_interval is 0. On the other
hand, the above codes mean that we don't send a WAL position request
and therefore never update oldest_nonremovable_xid if
wal_receiver_status_interval is 0. I'm concerned it could be a pitfall
for users.
---
% git show | grep update_delete
This set of patches aims to support the detection of
update_deleted conflicts,
transactions with earlier timestamps than the DELETE, detecting
update_delete
We assume that the appropriate resolution for update_deleted conflicts, to
that when detecting the update_deleted conflict, and the remote update has a
+ * to detect update_deleted conflicts.
+ * update_deleted is necessary, as the UPDATEs in remote transactions should be
+ * to allow for the detection of update_delete conflicts when applying
There are mixed 'update_delete' and 'update_deleted' in the commit
message and the codes. I think it's better to use 'update_deleted'.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wednesday, November 13, 2024 8:35 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I've started to review these patch series. I've reviewed only 0001 patch for now
but let me share some comments:
Thanks for the comment !
--- + if (*phase == DTR_WAIT_FOR_WALSENDER_WAL_POS) + { + Assert(xid_advance_attempt_time);What is this assertion for? If we want to check here that we have sent a request
message for the publisher, I think it's clearer if we have
"Assert(xid_advance_attempt_time > 0)". I'm not sure we really need this
assertion though since it's never false once we set xid_advance_attempt_time.
I agree that the assert is not useful, so removed in this version.
--- + /* + * Do not return here because the apply worker might have already + * applied all changes up to remote_lsn. Instead, proceed to the + * next phase to check if we can immediately advance the transaction + * ID. + */ + *phase = DTR_WAIT_FOR_LOCAL_FLUSH; + }If we always proceed to the next phase, is this phase really necessary? IIUC
even if we jump to DTR_WAIT_FOR_LOCAL_FLUSH phase after
DTR_REQUEST_WALSENDER_WAL_POS and have a check if we received the
remote WAL position in DTR_WAIT_FOR_LOCAL_FLUSH phase, it would work
fine.
Agreed. I removed this separate phase.
Although the DeadTupleRetainPhase enum only has 2 items now, I didn't remove
the enum in this version, as it could make the code easier to read and we may extent
the phases later if we accept the approach in 0001_2.
--- + /* + * 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, now we can advance the + * non-removable transaction ID. + */ + SpinLockAcquire(&MyLogicalRepWorker->relmutex); + MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid; + SpinLockRelease(&MyLogicalRepWorker->relmutex);How about adding a debug log message showing new
oldest_nonremovable_xid and related LSN for making the debug/investigation
easier? For example,elog(LOG, "confirmed remote flush up to %X/%X: new
oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(*remote_lsn),
XidFromFullTransactionId(candidate_xid));
Added.
--- + /* + * Exit early if the user has disabled sending messages to the + * publisher. + */ + if (wal_receiver_status_interval <= 0) + return;In send_feedback(), we send a feedback message if the publisher requests,
even if wal_receiver_status_interval is 0. On the other hand, the above codes
mean that we don't send a WAL position request and therefore never update
oldest_nonremovable_xid if wal_receiver_status_interval is 0. I'm concerned it
could be a pitfall for users.
It was intended to mirror the style of hot standby feedback (as implemented in
XLogWalRcvSendHSFeedback()) where the message is sent at most once per
wal_receiver_status_interval. I think the usage of the new message in the patch
is more similar to the hot standby feedback which are used to advance xmin of a
replication slot.
I also documented this risk in the detect_update_deleted option patch(0004).
Would it be sufficient to you ?
--- % git show | grep update_delete This set of patches aims to support the detection of update_deleted conflicts, transactions with earlier timestamps than the DELETE, detecting update_delete We assume that the appropriate resolution for update_deleted conflicts, to that when detecting the update_deleted conflict, and the remote update has a + * to detect update_deleted conflicts. + * update_deleted is necessary, as the UPDATEs in remote transactions should be + * to allow for the detection of update_delete conflicts when + applyingThere are mixed 'update_delete' and 'update_deleted' in the commit message
and the codes. I think it's better to use 'update_deleted'.
Thanks for searching. I have replaced them to update_deleted.
Attach the V9 patch set which addressed above comments.
Best Regards,
Hou zj
Attachments:
v9-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchapplication/octet-stream; name=v9-0004-Add-a-detect_update_deleted-option-to-subscriptio.patchDownload
From f3a8b791b7d06b0b284f92340ef172ef9956767d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 5 Nov 2024 09:55:02 +0800
Subject: [PATCH v9 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 52 +++++-
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 7 +-
src/backend/replication/logical/worker.c | 10 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 265 insertions(+), 94 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c180ed7abb..f00155ae65 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8042,6 +8042,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 1b63b58271..416fb0f78d 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1646,7 +1646,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..25d2c4ce55 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -563,7 +577,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -608,6 +623,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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1165,7 +1188,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,6 +1348,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 9e1f15bb6a..b8a83790cb 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_time = 0;
*delete_origin = InvalidRepOriginId;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index d77e38bf14..bc9a30007d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,6 +1190,8 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_dead_tuples |= sub->detectupdatedeleted;
+
if (!sub->enabled)
{
can_advance_xmin = false;
@@ -1253,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated = advance_conflict_slot_xmin(xmin);
@@ -1270,6 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f82677fafe..a0519b8119 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2743,7 +2743,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localts,
&localorigin) &&
localorigin != replorigin_session_origin)
@@ -3095,7 +3096,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localts,
&localorigin) &&
localorigin != replorigin_session_origin)
@@ -4032,6 +4034,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn)
Assert(remote_lsn);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index a8c141b689..d56e81cb0a 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4861,6 +4861,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4933,11 +4934,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5022,6 +5030,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5264,6 +5274,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 5bfebad64d..fc2288d114 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6602,7 +6602,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)
{
@@ -6670,6 +6670,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v9-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchapplication/octet-stream; name=v9-0001-Maintain-the-oldest-non-removeable-tranasction-ID.patchDownload
From 0d7f6ab023246ee2962b8684baf47e67a8283006 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v9] 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 and send a message to request the remote WAL position from
the walsender.
2) Wait to receive the WAL position from the walsender.
3) Advance the non-removable transaction ID if the current flush location
has reached or surpassed the received WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 62 ++++++
src/backend/replication/logical/worker.c | 227 +++++++++++++++++++++-
src/backend/replication/walsender.c | 47 +++++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 353 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..023b690a43 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2620,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..7c1a201002 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,18 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +353,10 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +396,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3593,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3716,42 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the initial phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3764,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3893,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3974,171 @@ 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:
+ *
+ * - DTR_REQUEST_WALSENDER_WAL_POS:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * After receiving the WAL position from the walsender, advance the
+ * non-removable transaction ID if the current flush location has reached or
+ * surpassed the received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_lsn' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_lsn);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_lsn))
+ return;
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(*remote_lsn),
+ XidFromFullTransactionId(candidate_xid));
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * WAL position request. Instead, proceed to the next phase to check
+ * if we can send the next request.
+ */
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ elog(DEBUG2, "sending WAL position request message");
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, now); /* sendTime */
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ *remote_lsn = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+}
+
/*
* 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 371eef3ddd..19d3b23f42 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,48 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ WalSnd *walsnd = MyWalSnd;
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * The latest WAL write position is only available on a non-standby
+ * server.
+ *
+ * XXX: We could consider forwarding the request to the primary server if
+ * the current server is a standby.
+ */
+ if (!RecoveryInProgress())
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending WAL write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..370c71c93e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 100afe40e1..4dcc778204 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v9-0001_2-Maintain-the-oldest-non-removeable-tranasction-I.patch.txttext/plain; name=v9-0001_2-Maintain-the-oldest-non-removeable-tranasction-I.patch.txtDownload
From e75c24c06a71d3e12e9fbd90be22af54fef73dea Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v99] 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 running
transactions.
3) Wait for the status from the walsender. After receiving the first status
after acquiring a new candidate transaction ID, do not proceed if there are
ongoing concurrent remote transactions. 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 89 ++++++
src/backend/replication/logical/worker.c | 314 +++++++++++++++++++++-
src/backend/replication/walsender.c | 57 ++++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 477 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..3d0cf32356 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,68 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest running transaction ID on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2647,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..48a37b3f51 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,20 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_GET_CANDIDATE_XID,
+ DTR_REQUEST_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +355,15 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+/* Buffers for constructing outgoing messages. */
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +403,11 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ TransactionId remote_oldestxid,
+ TransactionId remote_nextxid,
+ uint32 remote_epoch,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3603,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn;
+ TransactionId remote_oldestxid;
+ TransactionId remote_nextxid;
+ uint32 remote_epoch;
+ DeadTupleRetainPhase phase = DTR_GET_CANDIDATE_XID;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3729,50 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ remote_oldestxid = pq_getmsgint(&s, 4);
+ remote_nextxid = pq_getmsgint(&s, 4);
+ remote_epoch = pq_getmsgint(&s, 4);
+ timestamp = pq_getmsgint64(&s);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the request phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ maybe_advance_nonremovable_xid(&remote_lsn,
+ remote_oldestxid,
+ remote_nextxid,
+ remote_epoch,
+ &phase);
+
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3785,9 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, remote_oldestxid,
+ remote_nextxid, remote_epoch, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3915,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3996,236 @@ 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:
+ *
+ * - DTR_GET_CANDIDATE_XID:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid.
+ *
+ * - DTR_REQUEST_PUBLISHER_STATUS:
+ * Send a message to the walsender requesting the publisher status, which
+ * includes the latest WAL write position and information about running
+ * transactions.
+ *
+ * - DTR_WAIT_FOR_PUBLISHER_STATUS:
+ * Wait for the status from the walsender. After receiving the first status
+ * after acquiring a new candidate transaction ID, do not proceed if there
+ * are ongoing concurrent remote transactions. 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
+ * (DTR_REQUEST_PUBLISHER_STATUS) until all these transactions have
+ * completed.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * Advance the non-removable transaction ID if the current flush location has
+ * reached or surpassed the last received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_lsn' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ TransactionId remote_oldestxid,
+ TransactionId remote_nextxid,
+ uint32 remote_epoch,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+ static FullTransactionId next_phase_at;
+
+ TimestampTz now = 0;
+
+ Assert(remote_lsn);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_PUBLISHER_STATUS)
+ {
+ FullTransactionId remote_full_xid;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(next_phase_at))
+ next_phase_at = FullTransactionIdFromEpochAndXid(remote_epoch,
+ remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (remote_oldestxid > remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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.
+ *
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_lsn, or enough time might have
+ * passed. Instead, proceed to the next phase to check if we can
+ * immediately advance the transaction ID or send one more request.
+ */
+ if (FullTransactionIdPrecedesOrEquals(next_phase_at, remote_full_xid))
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ else
+ *phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_lsn) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(*remote_lsn),
+ XidFromFullTransactionId(candidate_xid));
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * transaction ID advancement. Instead, proceed to the next phase to
+ * check if we can get the next candidate transaction ID.
+ */
+ *phase = DTR_GET_CANDIDATE_XID;
+ }
+
+ Assert(*phase == DTR_GET_CANDIDATE_XID ||
+ *phase == DTR_REQUEST_PUBLISHER_STATUS);
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+
+ if (*phase == DTR_GET_CANDIDATE_XID)
+ {
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+ next_phase_at = InvalidFullTransactionId;
+
+ /*
+ * Do not return here to ensure an immediate request message is sent
+ * in the next phase.
+ */
+ *phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ if (*phase == DTR_REQUEST_PUBLISHER_STATUS)
+ {
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, now);
+
+ elog(DEBUG2, "sending publisher status request message");
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ *remote_lsn = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_PUBLISHER_STATUS;
+ }
+}
+
/*
* 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 371eef3ddd..63eed24c5a 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,57 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestRunningXid = InvalidTransactionId;
+ FullTransactionId nextFullXid = InvalidFullTransactionId;
+ WalSnd *walsnd = MyWalSnd;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * Information about running transactions and the WAL write position is
+ * only available on a non-standby server.
+ *
+ * XXX: We could consider forwarding the request to the primary server if
+ * the current server is a standby.
+ */
+ if (!RecoveryInProgress())
+ {
+ oldestRunningXid = GetOldestActiveTransactionId();
+ 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, oldestRunningXid);
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 100afe40e1..4dcc778204 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v9-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchapplication/octet-stream; name=v9-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mechan.patchDownload
From cdcae1803e9b4bd040dee3b7b5be21dde6ccb30e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v9 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 153 ++++++++++++++++++
2 files changed, 154 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..1470ed87d8
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,153 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', 'track_commit_timestamp = on');
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v9-0002-Maintain-the-replication-slot-in-logical-launcher.patchapplication/octet-stream; name=v9-0002-Maintain-the-replication-slot-in-logical-launcher.patchDownload
From e0c22c37ad41e3e6b32e4cf8a68cac4d0a42bf2d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v9 2/5] 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.
---
src/backend/replication/logical/launcher.c | 196 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 223 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..d77e38bf14 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
@@ -1215,6 +1248,30 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1299,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 19d3b23f42..576bfbd98d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v9-0003-Support-the-conflict-detection-for-update_deleted.patchapplication/octet-stream; name=v9-0003-Support-the-conflict-detection-for-update_deleted.patchDownload
From 00b2c9846ea2feafe5507477b84215b6b01661b0 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v9 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index b7e340824c..1b63b58271 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1639,6 +1639,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..9e1f15bb6a 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestXmin;
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ Bitmapset *indexbitmap = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 5ab4e02302..f82677fafe 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2687,6 +2687,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2703,15 +2706,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2722,7 +2721,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2741,19 +2740,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localts,
+ &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3075,7 +3082,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3085,17 +3092,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localts,
+ &localorigin) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3107,7 +3122,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3118,7 +3133,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..07b2c10892 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin);
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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
On Thu, Nov 14, 2024 at 8:24 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V9 patch set which addressed above comments.
Reviewed v9 patch-set and here are my comments for below changes:
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
+
+ if (can_advance_xmin && w != NULL)
+ {
+ FullTransactionId nonremovable_xid;
+
+ SpinLockAcquire(&w->relmutex);
+ nonremovable_xid = w->oldest_nonremovable_xid;
+ SpinLockRelease(&w->relmutex);
+
+ if (!FullTransactionIdIsValid(xmin) ||
+ !FullTransactionIdIsValid(nonremovable_xid) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin))
+ xmin = nonremovable_xid;
+ }
+
1) In Patch-0002, could you please add a comment above "+ if
(can_advance_xmin && w != NULL)" to briefly explain the purpose of
finding the minimum XID at this point?
2) In Patch-0004, with the addition of the 'detect_update_deleted'
option, I see the following two issues in the above code:
2a) Currently, all enabled subscriptions are considered when comparing
and finding the minimum XID, even if detect_update_deleted is disabled
for some subscriptions.
I suggest excluding the oldest_nonremovable_xid of subscriptions where
detect_update_deleted=false by updating the check as follows:
if (sub->detectupdatedeleted && can_advance_xmin && w != NULL)
2b) I understand why advancing xmin is not allowed when a subscription
is disabled. However, the current check allows a disabled subscription
with detect_update_deleted=false to block xmin advancement, which
seems incorrect. Should the check also account for
detect_update_deleted?, like :
if (sub->detectupdatedeleted && !sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
However, I'm not sure if this is the right fix, as it could lead to
inconsistencies if the detect_update_deleted is set to false after
disabling the sub.
Thoughts?
--
Thanks,
Nisha
On Wednesday, November 20, 2024 8:05 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
On Thu, Nov 14, 2024 at 8:24 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the V9 patch set which addressed above comments.
Reviewed v9 patch-set and here are my comments for below changes:
Thanks for the comments.
@@ -1175,10 +1189,29 @@ ApplyLauncherMain(Datum main_arg)
...
+ if (can_advance_xmin && w != NULL) + { + FullTransactionId nonremovable_xid; + + SpinLockAcquire(&w->relmutex); + nonremovable_xid = w->oldest_nonremovable_xid; + SpinLockRelease(&w->relmutex); + + if (!FullTransactionIdIsValid(xmin) || + !FullTransactionIdIsValid(nonremovable_xid) || + FullTransactionIdPrecedes(nonremovable_xid, xmin)) + xmin = nonremovable_xid; + } +1) In Patch-0002, could you please add a comment above "+ if
(can_advance_xmin && w != NULL)" to briefly explain the purpose of
finding the minimum XID at this point?
Sure, added.
2) In Patch-0004, with the addition of the 'detect_update_deleted'
option, I see the following two issues in the above code:
2a) Currently, all enabled subscriptions are considered when comparing
and finding the minimum XID, even if detect_update_deleted is disabled
for some subscriptions.
I suggest excluding the oldest_nonremovable_xid of subscriptions where
detect_update_deleted=false by updating the check as follows:if (sub->detectupdatedeleted && can_advance_xmin && w != NULL)
Right, that's a miss. Fixed in V10 patch set.
2b) I understand why advancing xmin is not allowed when a subscription
is disabled. However, the current check allows a disabled subscription
with detect_update_deleted=false to block xmin advancement, which
seems incorrect. Should the check also account for
detect_update_deleted?, like :
if (sub->detectupdatedeleted && !sub->enabled)
+ {
+ can_advance_xmin = false;
+ xmin = InvalidFullTransactionId;
continue;
+ }
Yes, I think we should add this check.
However, I'm not sure if this is the right fix, as it could lead to
inconsistencies if the detect_update_deleted is set to false after
disabling the sub.
Thoughts?
I think it's OK as it doesn't affect the functionality.
Attach the V10 patch set which addressed above comments
and fixed a CFbot warning due to un-initialized variable.
Best Regards,
Hou zj
Attachments:
v10_2-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v10_2-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 8cd073b304fe5d6221ed7b93b35c79cb52d264ec Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v10 1/5] 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 running
transactions.
3) Wait for the status from the walsender. After receiving the first status
after acquiring a new candidate transaction ID, do not proceed if there are
ongoing concurrent remote transactions. 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 89 ++++++
src/backend/replication/logical/worker.c | 314 +++++++++++++++++++++-
src/backend/replication/walsender.c | 57 ++++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 477 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..3d0cf32356 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,68 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest running transaction ID on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2647,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..54d5822fa1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,20 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_GET_CANDIDATE_XID,
+ DTR_REQUEST_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +355,15 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+/* Buffers for constructing outgoing messages. */
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +403,11 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ TransactionId remote_oldestxid,
+ TransactionId remote_nextxid,
+ uint32 remote_epoch,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3603,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn;
+ TransactionId remote_oldestxid;
+ TransactionId remote_nextxid;
+ uint32 remote_epoch;
+ DeadTupleRetainPhase phase = DTR_GET_CANDIDATE_XID;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3729,50 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ remote_oldestxid = pq_getmsgint(&s, 4);
+ remote_nextxid = pq_getmsgint(&s, 4);
+ remote_epoch = pq_getmsgint(&s, 4);
+ timestamp = pq_getmsgint64(&s);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the request phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ maybe_advance_nonremovable_xid(&remote_lsn,
+ remote_oldestxid,
+ remote_nextxid,
+ remote_epoch,
+ &phase);
+
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3785,9 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, remote_oldestxid,
+ remote_nextxid, remote_epoch, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3915,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3996,236 @@ 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:
+ *
+ * - DTR_GET_CANDIDATE_XID:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid.
+ *
+ * - DTR_REQUEST_PUBLISHER_STATUS:
+ * Send a message to the walsender requesting the publisher status, which
+ * includes the latest WAL write position and information about running
+ * transactions.
+ *
+ * - DTR_WAIT_FOR_PUBLISHER_STATUS:
+ * Wait for the status from the walsender. After receiving the first status
+ * after acquiring a new candidate transaction ID, do not proceed if there
+ * are ongoing concurrent remote transactions. 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
+ * (DTR_REQUEST_PUBLISHER_STATUS) until all these transactions have
+ * completed.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * Advance the non-removable transaction ID if the current flush location has
+ * reached or surpassed the last received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_lsn' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ TransactionId remote_oldestxid,
+ TransactionId remote_nextxid,
+ uint32 remote_epoch,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+ static FullTransactionId next_phase_at;
+
+ TimestampTz now = 0;
+
+ Assert(remote_lsn);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_PUBLISHER_STATUS)
+ {
+ FullTransactionId remote_full_xid;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(next_phase_at))
+ next_phase_at = FullTransactionIdFromEpochAndXid(remote_epoch,
+ remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (remote_oldestxid > remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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.
+ *
+ * Do not return here because the apply worker might have already
+ * applied all changes up to remote_lsn, or enough time might have
+ * passed. Instead, proceed to the next phase to check if we can
+ * immediately advance the transaction ID or send one more request.
+ */
+ if (FullTransactionIdPrecedesOrEquals(next_phase_at, remote_full_xid))
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ else
+ *phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ Assert(!XLogRecPtrIsInvalid(*remote_lsn) &&
+ FullTransactionIdIsValid(candidate_xid));
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(*remote_lsn),
+ XidFromFullTransactionId(candidate_xid));
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * transaction ID advancement. Instead, proceed to the next phase to
+ * check if we can get the next candidate transaction ID.
+ */
+ *phase = DTR_GET_CANDIDATE_XID;
+ }
+
+ Assert(*phase == DTR_GET_CANDIDATE_XID ||
+ *phase == DTR_REQUEST_PUBLISHER_STATUS);
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+
+ if (*phase == DTR_GET_CANDIDATE_XID)
+ {
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+ next_phase_at = InvalidFullTransactionId;
+
+ /*
+ * Do not return here to ensure an immediate request message is sent
+ * in the next phase.
+ */
+ *phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
+
+ if (*phase == DTR_REQUEST_PUBLISHER_STATUS)
+ {
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, now);
+
+ elog(DEBUG2, "sending publisher status request message");
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ *remote_lsn = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_PUBLISHER_STATUS;
+ }
+}
+
/*
* 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 371eef3ddd..63eed24c5a 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,57 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestRunningXid = InvalidTransactionId;
+ FullTransactionId nextFullXid = InvalidFullTransactionId;
+ WalSnd *walsnd = MyWalSnd;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * Information about running transactions and the WAL write position is
+ * only available on a non-standby server.
+ *
+ * XXX: We could consider forwarding the request to the primary server if
+ * the current server is a standby.
+ */
+ if (!RecoveryInProgress())
+ {
+ oldestRunningXid = GetOldestActiveTransactionId();
+ 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, oldestRunningXid);
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 08521d51a9..77a8674071 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v10-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v10-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From 657ad5c90f3a103b41a362a754c7c658bb278816 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 21 Nov 2024 12:39:40 +0800
Subject: [PATCH v10 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 52 +++++-
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 26 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 286 insertions(+), 108 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 59bb833f48..4667e1a3ad 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8043,6 +8043,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 1b63b58271..416fb0f78d 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1646,7 +1646,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..25d2c4ce55 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -162,6 +165,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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -307,6 +312,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_FAILOVER;
opts->failover = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = defGetBoolean(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -563,7 +577,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_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -608,6 +623,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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1165,7 +1188,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_DETECT_UPDATE_DELETED | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,6 +1348,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index ce911a41dc..13f29ea338 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8da897690a..5467a7ab21 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,12 +1190,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1201,11 +1211,12 @@ ApplyLauncherMain(Datum main_arg)
if (w != NULL)
{
/*
- * Collect non-removable transaction IDs from all apply workers
- * to determine the xmin for advancing the replication slot
+ * Collect non-removable transaction IDs from all apply
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication slot
* used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1263,7 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1285,6 +1296,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f4126a160c..4c6eb69fdf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2689,9 +2689,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2745,7 +2745,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3083,9 +3084,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3097,7 +3098,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3114,8 +3116,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4033,6 +4035,10 @@ maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
Assert(remote_lsn);
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index c30aafbe70..af3f930852 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4861,6 +4861,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4933,11 +4934,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5022,6 +5030,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5264,6 +5274,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 5bfebad64d..fc2288d114 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6602,7 +6602,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)
{
@@ -6670,6 +6670,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v10-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v10-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From ff4dd13265d54836335b2a85ec77fa3f6e5d6b00 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v10 1/5] 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 and send a message to request the remote WAL position from
the walsender.
2) Wait to receive the WAL position from the walsender.
3) Advance the non-removable transaction ID if the current flush location
has reached or surpassed the received WAL position.
These steps are repeated at intervals defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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 | 62 ++++++
src/backend/replication/logical/worker.c | 227 +++++++++++++++++++++-
src/backend/replication/walsender.c | 47 +++++
src/include/replication/worker_internal.h | 18 ++
src/tools/pgindent/typedefs.list | 1 +
5 files changed, 353 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..023b690a43 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,41 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2620,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..7c1a201002 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,18 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_REQUEST_WALSENDER_WAL_POS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +353,10 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
/* BufFile handle of the current streaming file */
static BufFile *stream_fd = NULL;
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -378,6 +396,8 @@ static void stream_open_and_write_change(TransactionId xid, char action, StringI
static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ DeadTupleRetainPhase *phase);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3573,6 +3593,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ XLogRecPtr remote_lsn = InvalidXLogRecPtr;
+ DeadTupleRetainPhase phase = DTR_REQUEST_WALSENDER_WAL_POS;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3694,6 +3716,42 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, false);
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ TimestampTz timestamp;
+
+ remote_lsn = pq_getmsgint64(&s);
+ timestamp = pq_getmsgint64(&s);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the initial phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+ UpdateWorkerStats(last_received, timestamp, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3764,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&remote_lsn, &phase);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3893,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3974,171 @@ 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:
+ *
+ * - DTR_REQUEST_WALSENDER_WAL_POS:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid and send a message to request the remote WAL position from
+ * the walsender.
+ *
+ * - DTR_WAIT_FOR_LOCAL_FLUSH:
+ * After receiving the WAL position from the walsender, advance the
+ * non-removable transaction ID if the current flush location has reached or
+ * surpassed the received WAL position.
+ *
+ * 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.
+ *
+ * The 'remote_lsn' will be reset after sending a new request to walsender.
+ */
+static void
+maybe_advance_nonremovable_xid(XLogRecPtr *remote_lsn,
+ DeadTupleRetainPhase *phase)
+{
+ static TimestampTz xid_advance_attempt_time = 0;
+ static FullTransactionId candidate_xid;
+
+ Assert(remote_lsn);
+
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ if (*phase == DTR_WAIT_FOR_LOCAL_FLUSH)
+ {
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(*remote_lsn))
+ return;
+
+ /*
+ * 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. In this case, confirming the
+ * apply and flush progress across all table sync workers is complex
+ * and not worth the effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < *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, now we can advance the
+ * non-removable transaction ID.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(*remote_lsn),
+ XidFromFullTransactionId(candidate_xid));
+
+ /*
+ * Do not return here as enough time might have passed since the last
+ * WAL position request. Instead, proceed to the next phase to check
+ * if we can send the next request.
+ */
+ *phase = DTR_REQUEST_WALSENDER_WAL_POS;
+ }
+
+ if (*phase == DTR_REQUEST_WALSENDER_WAL_POS)
+ {
+ TimestampTz now;
+ TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xid;
+ uint32 epoch;
+
+ /*
+ * Exit early if the user has disabled sending messages to the
+ * publisher.
+ */
+ if (wal_receiver_status_interval <= 0)
+ return;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ xid_advance_attempt_time = now;
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldestRunningXid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch,
+ oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ candidate_xid = full_xid;
+
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ elog(DEBUG2, "sending WAL position request message");
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, now); /* sendTime */
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ *remote_lsn = InvalidXLogRecPtr;
+ *phase = DTR_WAIT_FOR_LOCAL_FLUSH;
+ }
+}
+
/*
* 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 371eef3ddd..19d3b23f42 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -253,6 +253,7 @@ static void StartLogicalReplication(StartReplicationCmd *cmd);
static void ProcessStandbyMessage(void);
static void ProcessStandbyReplyMessage(void);
static void ProcessStandbyHSFeedbackMessage(void);
+static void ProcessStandbyWalPosRequestMessage(void);
static void ProcessRepliesIfAny(void);
static void ProcessPendingWrites(void);
static void WalSndKeepalive(bool requestReply, XLogRecPtr writePtr);
@@ -2314,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyWalPosRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2665,48 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyWalPosRequestMessage(void)
+{
+ WalSnd *walsnd = MyWalSnd;
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * The latest WAL write position is only available on a non-standby
+ * server.
+ *
+ * XXX: We could consider forwarding the request to the primary server if
+ * the current server is a standby.
+ */
+ if (!RecoveryInProgress())
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending WAL write position");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 08521d51a9..77a8674071 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -591,6 +591,7 @@ DbInfoArr
DbLocaleInfo
DeClonePtrType
DeadLockState
+DeadTupleRetainPhase
DeallocateStmt
DeclareCursorStmt
DecodedBkpBlock
--
2.30.0.windows.2
v10-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v10-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 91c642612d6f80c7228ae930158e0d694bd286c0 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v10 2/5] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..8da897690a 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,14 +1189,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1215,6 +1258,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1314,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 19d3b23f42..576bfbd98d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1189,6 +1189,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v10-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v10-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From 80917b11f0d9cbc851fa2542d7373383bb411836 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v10 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index b7e340824c..1b63b58271 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1639,6 +1639,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..ce911a41dc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 7c1a201002..f4126a160c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2689,6 +2689,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2705,15 +2708,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2724,7 +2723,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2743,19 +2742,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3077,7 +3084,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3087,17 +3094,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3109,7 +3124,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3120,7 +3135,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v10-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v10-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 4a03d468ddc6ea82740d49cb8f7424f23ac9f653 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v10 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 172 ++++++++++++++++++
2 files changed, 173 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..6b7d089c8c
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,172 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off});
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+$logfile = slurp_file($node_A->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=update_deleted.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+# 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 Node A");
+
+###############################################################################
+# Check that the replication slot pg_conflict_detection is dropped after
+# removing the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
On Thu, Nov 21, 2024 at 3:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V10 patch set which addressed above comments
and fixed a CFbot warning due to un-initialized variable.
We should make the v10_2-0001* as the first main patch for review till
we have a consensus to resolve LSN<->Timestamp inversion issue. This
is because v10_2 doesn't rely on the correctness of LSN<->Timestamp
mapping. Now, say in some later release, we fix the LSN<->Timestamp
inversion issue, we can simply avoid sending remote_xact information
and it will behave the same as your v10_1 approach.
Comments on v10_2_0001*:
======================
1.
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions between these phases.
+ */
+typedef enum
+{
+ DTR_GET_CANDIDATE_XID,
+ DTR_REQUEST_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_PUBLISHER_STATUS,
+ DTR_WAIT_FOR_LOCAL_FLUSH
+} DeadTupleRetainPhase;
First, can we have a better name for this enum like
RetainConflictInfoPhase or something like that? Second, the phase
transition is not very clear from the comments atop
maybe_advance_nonremovable_xid. You can refer to comments atop
tablesync.c or snapbuild.c to see other cases where we have explained
phase transitions.
2.
+ * Wait for the status from the walsender. After receiving the first status
+ * after acquiring a new candidate transaction ID, do not proceed if there
+ * are ongoing concurrent remote transactions.
In this part of the comments: " .. after acquiring a new candidate
transaction ID ..." appears misplaced.
3. In maybe_advance_nonremovable_xid(), the handling of each phase
looks ad-hoc though I see that you have done that have so that you can
handle the phase change functionality after changing the phase
immediately. If we have to ever extend this functionality, it will be
tricky to handle the new phase or at least the code will become
complicated. How about handling each phase in the order of their
occurrence and having separate functions for each phase as we have in
apply_dispatch()? That way it would be convenient to invoke the phase
handling functionality even if it needs to be called multiple times in
the same function.
4.
/*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ */
+ if (XLogRecPtrIsInvalid(remote_lsn))
+ {
+ ereport(WARNING,
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ /*
+ * Continuously revert to the request phase until
+ * the standby server (publisher) is promoted, at
+ * which point a valid WAL position will be
+ * received.
+ */
+ phase = DTR_REQUEST_PUBLISHER_STATUS;
+ }
Shouldn't this be an ERROR as the patch doesn't support this case? The
same should be true for later patches for the subscription option.
--
With Regards,
Amit Kapila.
On Monday, November 25, 2024 5:50 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Nov 21, 2024 at 3:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the V10 patch set which addressed above comments
and fixed a CFbot warning due to un-initialized variable.We should make the v10_2-0001* as the first main patch for review till
we have a consensus to resolve LSN<->Timestamp inversion issue. This
is because v10_2 doesn't rely on the correctness of LSN<->Timestamp
mapping. Now, say in some later release, we fix the LSN<->Timestamp
inversion issue, we can simply avoid sending remote_xact information
and it will behave the same as your v10_1 approach.
Agreed.
Comments on v10_2_0001*: ====================== 1. +/* + * The phases involved in advancing the non-removable transaction ID. + * + * Refer to maybe_advance_nonremovable_xid() for details on how the function + * transitions between these phases. + */ +typedef enum +{ + DTR_GET_CANDIDATE_XID, + DTR_REQUEST_PUBLISHER_STATUS, + DTR_WAIT_FOR_PUBLISHER_STATUS, + DTR_WAIT_FOR_LOCAL_FLUSH +} DeadTupleRetainPhase;First, can we have a better name for this enum like
RetainConflictInfoPhase or something like that? Second, the phase
transition is not very clear from the comments atop
maybe_advance_nonremovable_xid. You can refer to comments atop
tablesync.c or snapbuild.c to see other cases where we have explained
phase transitions.
I agree that RetainConflictInfoPhase is better so changed as suggested.
And I improve the comments a bit in the V11 patch set.
2. + * Wait for the status from the walsender. After receiving the first status + * after acquiring a new candidate transaction ID, do not proceed if there + * are ongoing concurrent remote transactions.In this part of the comments: " .. after acquiring a new candidate
transaction ID ..." appears misplaced.
Removed.
3. In maybe_advance_nonremovable_xid(), the handling of each phase
looks ad-hoc though I see that you have done that have so that you can
handle the phase change functionality after changing the phase
immediately. If we have to ever extend this functionality, it will be
tricky to handle the new phase or at least the code will become
complicated. How about handling each phase in the order of their
occurrence and having separate functions for each phase as we have in
apply_dispatch()? That way it would be convenient to invoke the phase
handling functionality even if it needs to be called multiple times in
the same function.
Agreed, I have split them into different functions.
4. /* + * An invalid position indiates the publisher is also + * a physical standby. In this scenario, advancing the + * non-removable transaction ID is not supported. 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. Refer to + * maybe_advance_nonremovable_xid() for details. + */ + if (XLogRecPtrIsInvalid(remote_lsn)) + { + ereport(WARNING, + errmsg("cannot get the latest WAL position from the publisher"), + errdetail("The connected publisher is also a standby server.")); + + /* + * Continuously revert to the request phase until + * the standby server (publisher) is promoted, at + * which point a valid WAL position will be + * received. + */ + phase = DTR_REQUEST_PUBLISHER_STATUS; + }Shouldn't this be an ERROR as the patch doesn't support this case? The
same should be true for later patches for the subscription option.
Yes, I changed the log level to ERROR in V11 patch set.
In V11, in addition to addressing above comments, I improved the speed of
advancing the non-removable transaction ID a bit:
In V10_2, the apply worker needs to wait for all the running transactions To
finish before advancing the non-removable transaction ID. That could delay the
advancement if there are long-running transactions on the publisher. I have
improved it to only wait for the transactions that might be in commit phase on
the publisher. I used the condition (DELAY_CHKPT_START & delayChkptFlags) to
determine whether a txn is in commit phase. Before f21bb9cfb564, that flag is
used for inCommit transactions as well which is similar to our situation. Ofc,
We could also add a new flag if necessary.
Also, I added a warning and comments for the case when
The clock on the publisher is behind that of the subscriber, as dead tuples
could be removed prematurely in this case.
Best Regards,
Hou zj
Attachments:
v11-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v11-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 12505e5dcd047decde15739edb2b5c875a1214c6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v11 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 172 ++++++++++++++++++
2 files changed, 173 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..8873f758d5
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,172 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off});
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+$logfile = slurp_file($node_A->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=update_deleted.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+# 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v11-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v11-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 8eb58f66015fca661693e57e3cea04fdd32f93cc Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v11] 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 running
transactions.
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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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/replication/logical/worker.c | 345 +++++++++++++++++++++-
src/backend/replication/walsender.c | 54 ++++
src/backend/storage/ipc/procarray.c | 59 ++++
src/include/replication/worker_internal.h | 18 ++
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
7 files changed, 567 insertions(+), 2 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..485dcc4ae5 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that might currently be in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2648,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..0cb7028e0a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,45 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions 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 might
+ * currently be 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 */
+ Timestamp xid_advance_attempt_time; /* when the candidate_xid is
+ * decided */
+ Timestamp reply_time; /* when the publisher responds with status */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +380,15 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+/* Buffers for constructing outgoing messages. */
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +429,12 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data);
+static void get_candidate_xid(RetainConflictInfoData *data);
+static void request_publisher_status(RetainConflictInfoData *data);
+static void wait_for_publisher_status(RetainConflictInfoData *data);
+static void wait_for_local_flush(RetainConflictInfoData *data);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3629,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData data;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3602,6 +3659,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
error_context_stack = &errcallback;
apply_error_context_stack = error_context_stack;
+ data.phase = RCI_GET_CANDIDATE_XID;
+
/* This outer loop iterates once per wait. */
for (;;)
{
@@ -3694,6 +3753,43 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, 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);
+
+ /*
+ * An invalid position indiates the publisher is also
+ * a physical standby. In this scenario, advancing the
+ * non-removable transaction ID is not supported. 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. Refer to
+ * maybe_advance_nonremovable_xid() for details.
+ *
+ * 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.
+ */
+ if (XLogRecPtrIsInvalid(data.remote_lsn))
+ ereport(ERROR,
+ errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
+ maybe_advance_nonremovable_xid(&data);
+
+ UpdateWorkerStats(last_received, data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3802,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&data);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3931,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +4012,251 @@ 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 running
+ * transactions.
+ *
+ * - 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 if concurrent remote transactions persist) ->
+ * WAIT_FOR_LOCAL_FLUSH.
+ *
+ * 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.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ 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);
+ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(data->xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->xid_advance_attempt_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ data->candidate_xid = full_xid;
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ request_publisher_status(data);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *data)
+{
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, GetCurrentTimestamp());
+
+ elog(DEBUG2, "sending publisher status request message");
+
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ data->remote_lsn = InvalidXLogRecPtr;
+ data->last_phase_at = InvalidFullTransactionId;
+ data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS;
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainConflictInfoData *data)
+{
+ FullTransactionId remote_full_xid;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(data->remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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;
+ wait_for_local_flush(data);
+ }
+ else
+ {
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+ request_publisher_status(data);
+ }
+}
+
+/*
+ * 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));
+
+ /*
+ * Issue a warning if there is a detected clock skew between the publisher
+ * and subscriber.
+ *
+ * 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->xid_advance_attempt_time, 0))
+ ereport(WARNING,
+ errmsg("non-removable 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* 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, now 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));
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+ get_candidate_xid(data);
+}
+
/*
* 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 371eef3ddd..d641aaeb1f 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,54 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit = InvalidTransactionId;
+ FullTransactionId nextFullXid = InvalidFullTransactionId;
+ WalSnd *walsnd = MyWalSnd;
+ TimestampTz replyTime;
+
+ 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);
+
+ /*
+ * Information about running transactions and the WAL write position is
+ * only available on a non-standby server.
+ */
+ if (!RecoveryInProgress())
+ {
+ 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 36610a1c7e..af0363e77b 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2929,6 +2929,65 @@ GetOldestActiveTransactionId(void)
return oldestRunningXid;
}
+
+/*
+ * GetOldestTransactionIdInCommit()
+ *
+ * Similar to GetOldestActiveTransactionId but returns the oldest transaction ID
+ * that might currently be 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_START) != 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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 56af0b40b3..c388eec6a5 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 b54428b38c..da7a859a7b 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2469,6 +2469,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v11-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v11-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 6866656b8994f28fe31a531c5475434fd2c69050 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v11 2/4] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..8da897690a 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,14 +1189,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1215,6 +1258,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1314,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index d641aaeb1f..78eb1dbb3d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v11-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v11-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From afa5d3edbfd64695ac1664fed766477f49030e74 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v11 3/4] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..41ae3e2a07 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1640,6 +1640,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..ce911a41dc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 d0f7ece1d4..1e6b5b9297 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,6 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2740,15 +2743,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2759,7 +2758,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2778,19 +2777,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3112,7 +3119,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3122,17 +3129,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3144,7 +3159,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3155,7 +3170,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v11-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v11-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From 9160f52331a95b3120be05e974aa8d4cfe8f3a69 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 26 Nov 2024 12:01:13 +0800
Subject: [PATCH v11 4/4] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 114 +++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 26 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 342 insertions(+), 114 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 59bb833f48..4667e1a3ad 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8043,6 +8043,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..8e58ddb20c 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1568,14 +1619,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1596,9 +1647,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
PG_TRY();
{
- walrcv_alter_slot(wrconn, sub->slotname,
- update_failover ? &opts.failover : NULL,
- update_two_phase ? &opts.twophase : NULL);
+ if (query_remote_recovery)
+ check_remote_recovery(wrconn, opts.detectupdatedeleted);
+
+ 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();
{
@@ -2193,6 +2248,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * operating as a standby server and report an ERROR in such cases.
+ *
+ * Refer to the comments above the invocation of
+ * maybe_advance_nonremovable_xid() for detailed reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 primary server: %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 detect_update_deleted option if the publisher is also a standby server."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index ce911a41dc..13f29ea338 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8da897690a..5467a7ab21 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,12 +1190,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1201,11 +1211,12 @@ ApplyLauncherMain(Datum main_arg)
if (w != NULL)
{
/*
- * Collect non-removable transaction IDs from all apply workers
- * to determine the xmin for advancing the replication slot
+ * Collect non-removable transaction IDs from all apply
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication slot
* used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1263,7 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1285,6 +1296,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 1e6b5b9297..8092fec3d1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,9 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2780,7 +2780,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3118,9 +3119,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3132,7 +3133,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3149,8 +3151,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4078,6 +4080,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index add7f16c90..19c50e9ea6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4982,6 +4989,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5028,6 +5036,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5270,6 +5280,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 5bfebad64d..fc2288d114 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6602,7 +6602,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)
{
@@ -6670,6 +6670,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
On Tue, Nov 26, 2024 at 1:50 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Few comments on the latest 0001 patch:
1.
+ * - RCI_REQUEST_PUBLISHER_STATUS:
+ * Send a message to the walsender requesting the publisher status, which
+ * includes the latest WAL write position and information about running
+ * transactions.
Shall we make the later part of this comment (".. information about
running transactions.") accurate w.r.t the latest changes of
requesting xacts that are known to be in the process of committing?
2.
+ * The overall state progression is: GET_CANDIDATE_XID ->
+ * REQUEST_PUBLISHER_STATUS -> WAIT_FOR_PUBLISHER_STATUS -> (loop to
+ * REQUEST_PUBLISHER_STATUS if concurrent remote transactions persist) ->
+ * WAIT_FOR_LOCAL_FLUSH.
This state machine progression misses to mention that after we waited
for flush the state again moves back to GET_CANDIDATE_XID.
3.
+request_publisher_status(RetainConflictInfoData *data)
+{
...
+ /* Send a WAL position request message to the server */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
This message requests more than a WAL write position but the comment
is incomplete.
4.
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
...
+ /*
+ * Information about running transactions and the WAL write position is
+ * only available on a non-standby server.
+ */
+ if (!RecoveryInProgress())
+ {
+ oldestXidInCommit = GetOldestTransactionIdInCommit();
+ nextFullXid = ReadNextFullTransactionId();
+ lsn = GetXLogWriteRecPtr();
+ }
Shall we ever reach here for a standby case? If not shouldn't that be an ERROR?
--
With Regards,
Amit Kapila.
On Wednesday, November 27, 2024 6:56 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Nov 26, 2024 at 1:50 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Few comments on the latest 0001 patch: 1. + * - RCI_REQUEST_PUBLISHER_STATUS: + * Send a message to the walsender requesting the publisher status, which + * includes the latest WAL write position and information about running + * transactions.Shall we make the later part of this comment (".. information about running
transactions.") accurate w.r.t the latest changes of requesting xacts that are
known to be in the process of committing?
Changed.
2. + * The overall state progression is: GET_CANDIDATE_XID -> + * REQUEST_PUBLISHER_STATUS -> WAIT_FOR_PUBLISHER_STATUS -> (loop to + * REQUEST_PUBLISHER_STATUS if concurrent remote transactions persist) + -> + * WAIT_FOR_LOCAL_FLUSH.This state machine progression misses to mention that after we waited for
flush the state again moves back to GET_CANDIDATE_XID.
Added.
3. +request_publisher_status(RetainConflictInfoData *data) { ... + /* Send a WAL position request message to the server */ + walrcv_send(LogRepWorkerWalRcvConn, + reply_message->data, reply_message->len);This message requests more than a WAL write position but the comment is
incomplete.
Thanks for pointing, improved.
4. +/* + * Process the request for a primary status update message. + */ +static void +ProcessStandbyPSRequestMessage(void) ... + /* + * Information about running transactions and the WAL write position is + * only available on a non-standby server. + */ + if (!RecoveryInProgress()) + { + oldestXidInCommit = GetOldestTransactionIdInCommit(); nextFullXid = + ReadNextFullTransactionId(); lsn = GetXLogWriteRecPtr(); }Shall we ever reach here for a standby case? If not shouldn't that be an ERROR?
It is possible to reach here if user creates a subscription with
(connect=false,detect_update_deleted=true), in which case we could not check it
during creation. But I agree that it would be better to report an ERROR here,
so changed as suggested. After this change, there is no need to check the
invalid remote lsn in apply worker and thus the error can also be removed.
I also modified the 0004 patch to check the remote server's recovery during
"alter subscription enable and alter subscription connect", so that it can avoid
sending a request to the standby in apply worker.
Attached is the V12 patch set.
In addition to addressing the previous comments, I have introduced a new flag,
DELAY_CHKPT_IN_COMMIT, within delayChkptFlags. This flag is marked in
RecordTransactionCommit(). This is because the existing DELAY_CHKPT_START is
used in other places such as PREPARE where we don't have to wait, so relying on
it might delay the advancement of non-removable xid due to these un-related
txns.
Best Regards,
Hou zj
Attachments:
v12-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v12-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From ab68e12cc49452c6aed6761782ea2cf02dbb3a72 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v12 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v12-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v12-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 6354d1b9cf96e9d647bb94ababd8c062d3b51a06 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v12 1/3] 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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/xact.c | 6 +-
src/backend/replication/logical/worker.c | 331 +++++++++++++++++++++-
src/backend/replication/walsender.c | 50 ++++
src/backend/storage/ipc/procarray.c | 59 ++++
src/include/replication/worker_internal.h | 18 ++
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
9 files changed, 557 insertions(+), 5 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index d5a78694b9..0046c11931 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2441,6 +2441,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2585,6 +2648,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index b7ebcc2a55..f41a9868f6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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/worker.c b/src/backend/replication/logical/worker.c
index 925dff9cc4..f0778dae7b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,44 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * Refer to maybe_advance_nonremovable_xid() for details on how the function
+ * transitions 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 */
+ Timestamp xid_advance_attempt_time; /* when the candidate_xid is
+ * decided */
+ Timestamp reply_time; /* when the publisher responds with status */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +379,15 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
+/* Buffers for constructing outgoing messages. */
+static StringInfo reply_message = NULL;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +428,12 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data);
+static void get_candidate_xid(RetainConflictInfoData *data);
+static void request_publisher_status(RetainConflictInfoData *data);
+static void wait_for_publisher_status(RetainConflictInfoData *data);
+static void wait_for_local_flush(RetainConflictInfoData *data);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData data;
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3602,6 +3658,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
error_context_stack = &errcallback;
apply_error_context_stack = error_context_stack;
+ data.phase = RCI_GET_CANDIDATE_XID;
+
/* This outer loop iterates once per wait. */
for (;;)
{
@@ -3694,6 +3752,18 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, reply_requested, 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);
+
+ maybe_advance_nonremovable_xid(&data);
+
+ UpdateWorkerStats(last_received, data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3776,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&data);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3833,12 +3905,10 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
static void
send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
{
- static StringInfo reply_message = NULL;
static TimestampTz send_time = 0;
static XLogRecPtr last_recvpos = InvalidXLogRecPtr;
static XLogRecPtr last_writepos = InvalidXLogRecPtr;
- static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
XLogRecPtr writepos;
XLogRecPtr flushpos;
@@ -3916,6 +3986,263 @@ 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 if concurrent remote transactions persist) ->
+ * 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)
+{
+ 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);
+ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and send a message at most once per
+ * wal_receiver_status_interval.
+ */
+ if (!TimestampDifferenceExceeds(data->xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->xid_advance_attempt_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ data->candidate_xid = full_xid;
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ request_publisher_status(data);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *data)
+{
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, GetCurrentTimestamp());
+
+ elog(DEBUG2, "sending publisher status request message");
+
+ /* Send a request for the publisher status */
+ walrcv_send(LogRepWorkerWalRcvConn,
+ reply_message->data, reply_message->len);
+
+ data->remote_lsn = InvalidXLogRecPtr;
+ data->last_phase_at = InvalidFullTransactionId;
+ data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS;
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainConflictInfoData *data)
+{
+ FullTransactionId remote_full_xid;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(data->remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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;
+ wait_for_local_flush(data);
+ }
+ else
+ {
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+ request_publisher_status(data);
+ }
+}
+
+/*
+ * 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));
+
+ /*
+ * Issue a warning if there is a detected clock skew between the publisher
+ * and subscriber.
+ *
+ * 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->xid_advance_attempt_time, 0))
+ ereport(WARNING,
+ errmsg("non-removable 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* 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, now 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));
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+ get_candidate_xid(data);
+}
+
/*
* 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 371eef3ddd..00b6411c7e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,50 @@ 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;
+
+ 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 36610a1c7e..8feed1d064 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2929,6 +2929,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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 1847bbfa95..349671c1db 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2466,6 +2466,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.31.1
v12-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v12-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From b8a3ea39d3c7e2c6f1c3c98ba16fd980e87aed05 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v12 2/3] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e5fdca8bbf..aeb51dacd7 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1127,6 +1136,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1155,6 +1167,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();
@@ -1175,14 +1189,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1215,6 +1258,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1242,6 +1314,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 6828100cf1..7d6b74c39f 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..8c18264f58 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 45582cf9d8..0cfcbb38a0 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.
*
@@ -258,6 +265,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.31.1
v12-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v12-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From a5cfd1dc81c50d35959a461baeba27808685a59f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v12 3/3] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index b7e340824c..1b63b58271 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1639,6 +1639,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..ce911a41dc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 f0778dae7b..7eb919972a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,6 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2740,15 +2743,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2759,7 +2758,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2778,19 +2777,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3112,7 +3119,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3122,17 +3129,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3144,7 +3159,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3155,7 +3170,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.31.1
v12-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v12-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From 5d3e9680094f55bf0fc53e60283b3f8558d80aa5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 26 Nov 2024 12:01:13 +0800
Subject: [PATCH v12 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 26 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 363 insertions(+), 117 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 59bb833f48..4667e1a3ad 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8043,6 +8043,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index ce911a41dc..13f29ea338 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8da897690a..5467a7ab21 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1168,6 +1169,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1188,12 +1190,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1201,11 +1211,12 @@ ApplyLauncherMain(Datum main_arg)
if (w != NULL)
{
/*
- * Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * Collect non-removable transaction IDs from all apply
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1263,7 +1274,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1285,6 +1296,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 85d7671918..b4b2257209 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2725,9 +2725,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2781,7 +2781,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3119,9 +3120,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3133,7 +3134,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3150,8 +3152,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4067,6 +4069,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index add7f16c90..19c50e9ea6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4982,6 +4989,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5028,6 +5036,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5270,6 +5280,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 5bfebad64d..fc2288d114 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6602,7 +6602,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)
{
@@ -6670,6 +6670,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
On Fri, Nov 29, 2024 at 7:54 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
It is possible to reach here if user creates a subscription with
(connect=false,detect_update_deleted=true), in which case we could not check it
during creation. But I agree that it would be better to report an ERROR here,
so changed as suggested. After this change, there is no need to check the
invalid remote lsn in apply worker and thus the error can also be removed.
1.
if (XLogRecPtrIsInvalid(data.remote_lsn))
+ ereport(ERROR,
+ errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("cannot get the latest WAL position from the publisher"),
+ errdetail("The connected publisher is also a standby server."));
+
Instead of removing this message from the patch, we should change it
to elog(ERROR, category of ERROR.
2.
+ Timestamp xid_advance_attempt_time; /* when the candidate_xid is
+ * decided */
How about naming this variable as candidate_xid_time /* time when the
next candidate_xid is computed */?
3.
+ /* Return if the new transaction ID is unchanged */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
This comment is unclear. Can we change it to: "Return if the
oldest_nonremovable_xid can't be advanced" or something like that?
4.
+request_publisher_status(RetainConflictInfoData *data)
+{
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
+
+ pq_sendbyte(reply_message, 'S');
+ pq_sendint64(reply_message, GetCurrentTimestamp());
+
+ elog(DEBUG2, "sending publisher status request message");
The name 'reply_message' sounds confusing as this is a request
message. Can we change it to request_message? Also, let's avoid
reusing the same variable among different messages as it makes the
code unclear.
Apart from the above, I have modified a few comments after applying
0001 and 0002 in the attached.
--
With Regards,
Amit Kapila.
Attachments:
v12_diff_amit_1.patch.txttext/plain; charset=US-ASCII; name=v12_diff_amit_1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 07e9916a1b..75fd3a5f7f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -280,8 +280,8 @@ typedef enum
/*
* The phases involved in advancing the non-removable transaction ID.
*
- * Refer to maybe_advance_nonremovable_xid() for details on how the function
- * transitions between these phases.
+ * See maybe_advance_nonremovable_xid() for details of the transition
+ * between these phases.
*/
typedef enum
{
@@ -385,7 +385,7 @@ static BufFile *stream_fd = NULL;
*/
static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
-/* Buffers for constructing outgoing messages. */
+/* Buffer for constructing outgoing messages. */
static StringInfo reply_message = NULL;
typedef struct SubXactInfo
@@ -4082,8 +4082,12 @@ get_candidate_xid(RetainConflictInfoData *data)
now = GetCurrentTimestamp();
/*
- * Compute the candidate_xid and send a message at most once per
- * wal_receiver_status_interval.
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and network
+ * resources without making much progress.
+ *
+ * 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.
*/
if (!TimestampDifferenceExceeds(data->xid_advance_attempt_time, now,
wal_receiver_status_interval * 1000))
Dear Hou,
Thanks for updating the patch! Here are my comments mainly for 0001.
01. protocol.sgml
I think the ordering of attributes in "Primary status update" seems not correct.
The second entry is LSN, not the oldest running xid.
02. maybe_advance_nonremovable_xid
```
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(data);
+ break;
```
I think the part is not reachable because the transit
RCI_REQUEST_PUBLISHER_STATUS->RCI_WAIT_FOR_PUBLISHER_STATUS is done in
get_candidate_xid()->request_publisher_status().
Can we remove this?
03. RetainConflictInfoData
```
+ Timestamp xid_advance_attempt_time; /* when the candidate_xid is
+ * decided */
+ Timestamp reply_time; /* when the publisher responds with status */
+
+} RetainConflictInfoData;
```
The datatype should be TimestampTz.
04. get_candidate_xid
```
+ if (!TimestampDifferenceExceeds(data->xid_advance_attempt_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
```
I think data->xid_advance_attempt_time can be accessed without the initialization
at the first try. I've found the patch could not pass test for 32-bit build
due to the reason.
05. request_publisher_status
```
+ if (!reply_message)
+ {
+ MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext);
+
+ reply_message = makeStringInfo();
+ MemoryContextSwitchTo(oldctx);
+ }
+ else
+ resetStringInfo(reply_message);
```
Same lines exist in two functions: can we provide an inline function?
06. wait_for_publisher_status
```
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
```
Not sure, is there a possibility that data->last_phase_at is valid here? It is initialized
just before transiting to RCI_WAIT_FOR_PUBLISHER_STATUS.
07. wait_for_publisher_status
I think all calculations and checking in the function can be done even on the
walsender. Based on this, I come up with an idea to reduce the message size:
walsender can just send a status (boolean) whether there are any running transactions
instead of oldest xid, next xid and their epoch. Or, it is more important to reduce the
amount of calc. on publisher side?
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Fri, Nov 29, 2024 at 4:05 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:
07. wait_for_publisher_status
I think all calculations and checking in the function can be done even on the
walsender. Based on this, I come up with an idea to reduce the message size:
walsender can just send a status (boolean) whether there are any running transactions
instead of oldest xid, next xid and their epoch. Or, it is more important to reduce the
amount of calc. on publisher side?
Won't it be tricky to implement this tracking on publisher side?
Because we not only need to check that there is no running xact but
also that the oldest_running_xact that was present last time when the
status message arrived has finished. Won't this need more bookkeeping
on publisher's side?
--
With Regards,
Amit Kapila.
On Fri, Nov 29, 2024 at 4:05 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:
02. maybe_advance_nonremovable_xid
``` + case RCI_REQUEST_PUBLISHER_STATUS: + request_publisher_status(data); + break; ```I think the part is not reachable because the transit
RCI_REQUEST_PUBLISHER_STATUS->RCI_WAIT_FOR_PUBLISHER_STATUS is done in
get_candidate_xid()->request_publisher_status().
Can we remove this?
After changing phase to RCI_REQUEST_PUBLISHER_STATUS, we directly
invoke request_publisher_status, and similarly, after changing phase
to RCI_WAIT_FOR_LOCAL_FLUSH, we call wait_for_local_flush. Won't it be
better that in both cases and other similar cases, we instead invoke
maybe_advance_nonremovable_xid()? This will make
maybe_advance_nonremovable_xid() the only function with the knowledge
to take action based on phase rather than spreading the knowledge of
phase-related actions to various functions. Then we should also add a
comment at the end in request_publisher_status() where we change the
phase but don't do anything. The comment should explain the reason for
the same.
One more point, it seems on a busy server, the patch won't be able to
advance nonremovable_xid. We should call
maybe_advance_nonremovable_xid() at all the places where we call
send_feedback() and additionally, we should also call it after
applying some threshold number (say 100) of messages. The latter is to
avoid the cases where we won't invoke the required functionality on a
busy server with a large value of sender/receiver timeouts.
--
With Regards,
Amit Kapila.
On Monday, December 2, 2024 12:14 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Nov 29, 2024 at 4:05 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:02. maybe_advance_nonremovable_xid
``` + case RCI_REQUEST_PUBLISHER_STATUS: + request_publisher_status(data); + break; ```I think the part is not reachable because the transit
RCI_REQUEST_PUBLISHER_STATUS->RCI_WAIT_FOR_PUBLISHER_STATU
S is done inget_candidate_xid()->request_publisher_status().
Can we remove this?After changing phase to RCI_REQUEST_PUBLISHER_STATUS, we directly
invoke request_publisher_status, and similarly, after changing phase to
RCI_WAIT_FOR_LOCAL_FLUSH, we call wait_for_local_flush. Won't it be
better that in both cases and other similar cases, we instead invoke
maybe_advance_nonremovable_xid()? This will make
maybe_advance_nonremovable_xid() the only function with the knowledge to
take action based on phase rather than spreading the knowledge of
phase-related actions to various functions. Then we should also add a
comment at the end in request_publisher_status() where we change the
phase but don't do anything. The comment should explain the reason for the
same.
Agreed.
One more point, it seems on a busy server, the patch won't be able to advance
nonremovable_xid. We should call
maybe_advance_nonremovable_xid() at all the places where we call
send_feedback() and additionally, we should also call it after applying some
threshold number (say 100) of messages. The latter is to avoid the cases where
we won't invoke the required functionality on a busy server with a large value of
sender/receiver timeouts.
Right. I think instead of adding a threshold, we could directly check if it's
time to start next round of xid advancement attempt. Since we already get the
current timestamp in the loop of receiving msg, it would not be expensive to
check the time difference. I have used this approach in V13 and will test the
performance for it.
Here is the V13 patch set which addressed this and pending comments
in [1]/messages/by-id/TYAPR01MB5692A6AC6EF2E22A2C9D099AF52A2@TYAPR01MB5692.jpnprd01.prod.outlook.com[2]/messages/by-id/CAA4eK1L-J-iJJ6gpzUc9wJY6eGfBbCDmh+5fHUZoixnoFbJSNg@mail.gmail.com.
[1]: /messages/by-id/TYAPR01MB5692A6AC6EF2E22A2C9D099AF52A2@TYAPR01MB5692.jpnprd01.prod.outlook.com
[2]: /messages/by-id/CAA4eK1L-J-iJJ6gpzUc9wJY6eGfBbCDmh+5fHUZoixnoFbJSNg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v13-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v13-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From 555fc893456e97f7bfd8f4b6fa41117c8b661dbd Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v13 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..41ae3e2a07 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1640,6 +1640,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..ce911a41dc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 3e9dd292cb..60e264530a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,6 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2740,15 +2743,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2759,7 +2758,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2778,19 +2777,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3112,7 +3119,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3122,17 +3129,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3144,7 +3159,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3155,7 +3170,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v13-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v13-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From e58a908de2d923c4b0edc078004176ea3e748883 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v13 1/5] 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 380 ++++++++++++++++++++-
src/backend/replication/walsender.c | 50 +++
src/backend/storage/ipc/procarray.c | 59 ++++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 608 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index cff0c4099e..d8c98ead23 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2442,6 +2442,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2586,6 +2649,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1eccb78ddc..d48ddc83af 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 e5fdca8bbf..16f26866f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -450,6 +450,7 @@ retry:
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 925dff9cc4..3e9dd292cb 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,45 @@ 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 candidate_xid_time; /* when the candidate_xid is decided */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ TimestampTz next_attempt_time; /* when to attemp to advance the xid during
+ * change application */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +380,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +426,14 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data);
+static void get_candidate_xid(RetainConflictInfoData *data);
+static void request_publisher_status(RetainConflictInfoData *data);
+static void wait_for_publisher_status(RetainConflictInfoData *data);
+static void wait_for_local_flush(RetainConflictInfoData *data);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *data,
+ TimestampTz now);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3677,6 +3733,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * during change application to prevent it from
+ * remaining unchanged for long periods when the worker
+ * is busy.
+ */
+ if (can_advance_nonremovable_xid(&data, last_recv_timestamp))
+ maybe_advance_nonremovable_xid(&data);
}
else if (c == 'k')
{
@@ -3692,8 +3757,26 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&data);
+
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);
+
+ if (XLogRecPtrIsInvalid(data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&data);
+
+ UpdateWorkerStats(last_received, data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3789,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&data);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3803,6 +3888,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&data);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3838,7 +3925,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;
@@ -3916,6 +4002,298 @@ 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 if concurrent remote transactions persist) ->
+ * 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)
+{
+ 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);
+ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and network
+ * resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ data->candidate_xid = full_xid;
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * 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);
+
+ pq_sendbyte(request_message, 'S');
+ 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->remote_lsn = InvalidXLogRecPtr;
+ data->last_phase_at = InvalidFullTransactionId;
+ data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS;
+
+ /*
+ * Skip calling maybe_advance_nonremovable_xid() since further actions
+ * cannot proceed until the publisher status is received.
+ */
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainConflictInfoData *data)
+{
+ FullTransactionId remote_full_xid;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(data->remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *data)
+{
+ XLogRecPtr writepos;
+ XLogRecPtr flushpos;
+ bool have_pending_txes;
+
+ Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
+ FullTransactionIdIsValid(data->candidate_xid));
+
+ /*
+ * Issue a warning if there is a detected clock skew between the publisher
+ * and subscriber.
+ *
+ * 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(WARNING,
+ errmsg("non-removable 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Fetch the latest flush position */
+ get_flush_position(&writepos, &flushpos, &have_pending_txes);
+
+ if (flushpos > last_flushpos)
+ last_flushpos = flushpos;
+
+ /* 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, now 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));
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * Determine if the next round of transaction ID advancement can be attempted.
+ *
+ * TODO: The remote flush location (last_flushpos) is currently not updated
+ * during change application, making it impossible to satisfy the condition of
+ * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
+ * Consider updating the remote flush position in the final phase to enable
+ * advancement during change application.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000);
+}
+
/*
* 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 371eef3ddd..00b6411c7e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,50 @@ 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;
+
+ 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 36610a1c7e..8feed1d064 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2929,6 +2929,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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 2d4c870423..c887c39a0a 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2469,6 +2469,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v13-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v13-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From ff581493da838ffb20c10216fcebe972a8d36354 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 2 Dec 2024 19:03:51 +0800
Subject: [PATCH v13 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 29 ++--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 364 insertions(+), 117 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 59bb833f48..4667e1a3ad 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8043,6 +8043,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index ce911a41dc..13f29ea338 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 07d0ca477d..ecd11e9508 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 60e264530a..37b0adfed6 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,9 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2780,7 +2780,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3118,9 +3119,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3132,7 +3133,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3149,8 +3151,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4081,6 +4083,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4304,7 +4310,8 @@ wait_for_local_flush(RetainConflictInfoData *data)
static bool
can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
{
- return data->phase == RCI_GET_CANDIDATE_XID &&
+ return MySubscription->detectupdatedeleted &&
+ data->phase == RCI_GET_CANDIDATE_XID &&
TimestampDifferenceExceeds(data->candidate_xid_time, now,
wal_receiver_status_interval * 1000);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index add7f16c90..19c50e9ea6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4982,6 +4989,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5028,6 +5036,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5270,6 +5280,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 0aa39906a1..01e36acc2c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6643,7 +6643,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)
{
@@ -6711,6 +6711,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v13-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v13-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 9c77276c5713bebe34ccc6b36490ea30e989ed3f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v13 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v13-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v13-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 1572a677376a927be633d11a0d6b5c1d84ac8d2c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v13 2/5] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 16f26866f2..07d0ca477d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1128,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1156,6 +1168,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();
@@ -1176,14 +1190,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1216,6 +1259,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1243,6 +1315,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 887e38d56e..5cb8286d36 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..8c18264f58 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
On Friday, November 29, 2024 6:35 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch! Here are my comments mainly for 0001.
Thanks for the comments!
02. maybe_advance_nonremovable_xid
``` + case RCI_REQUEST_PUBLISHER_STATUS: + request_publisher_status(data); + break; ```I think the part is not reachable because the transit
RCI_REQUEST_PUBLISHER_STATUS->RCI_WAIT_FOR_PUBLISHER_STATU
S is done in get_candidate_xid()->request_publisher_status().
Can we remove this?
I changed to call the maybe_advance_nonremovable_xid() after changing the phase
in get_candidate_xid/wait_for_publisher_status, so that the code is reachable.
05. request_publisher_status
``` + if (!reply_message) + { + MemoryContext oldctx = MemoryContextSwitchTo(ApplyContext); + + reply_message = makeStringInfo(); + MemoryContextSwitchTo(oldctx); + } + else + resetStringInfo(reply_message); ```Same lines exist in two functions: can we provide an inline function?
I personally feel these codes may not worth a separate function since it’s simple.
So didn't change in this version.
06. wait_for_publisher_status
``` + if (!FullTransactionIdIsValid(data->last_phase_at)) + data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch, + + data->remote_nextxid); + ```Not sure, is there a possibility that data->last_phase_at is valid here? It is
initialized just before transiting to RCI_WAIT_FOR_PUBLISHER_STATUS.
Oh. I think last_phase_at should be initialized only in the first phase. Fixed.
Other comments look good to me and have been addressed in V13.
Best Regards,
Hou zj
On Monday, December 2, 2024 7:34 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
Here is the V13 patch set which addressed this and pending comments in
[1][2].
I am sorry that I missed to merge some changes in previous patch set. Please check
the attached updated patch set V13_2.
[1]
/messages/by-id/TYAPR01MB5692A6AC6EF2E22A2C
9D099AF52A2%40TYAPR01MB5692.jpnprd01.prod.outlook.com
[2]
/messages/by-id/CAA4eK1L-J-iJJ6gpzUc9wJY6eGfB
bCDmh%2B5fHUZoixnoFbJSNg%40mail.gmail.com
Best Regards,
Hou zj
Attachments:
v13_2-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v13_2-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From b22cb40948d2f77dd386a0cca45f7b75eb871df3 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v13] 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 380 ++++++++++++++++++++-
src/backend/replication/walsender.c | 50 +++
src/backend/storage/ipc/procarray.c | 59 ++++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 608 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index cff0c4099e..d8c98ead23 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2442,6 +2442,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2586,6 +2649,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1eccb78ddc..d48ddc83af 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 e5fdca8bbf..16f26866f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -450,6 +450,7 @@ retry:
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 925dff9cc4..9b7f312eb0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,45 @@ 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 candidate_xid_time; /* when the candidate_xid is decided */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ TimestampTz next_attempt_time; /* when to attemp to advance the xid during
+ * change application */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +380,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +426,14 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data);
+static void get_candidate_xid(RetainConflictInfoData *data);
+static void request_publisher_status(RetainConflictInfoData *data);
+static void wait_for_publisher_status(RetainConflictInfoData *data);
+static void wait_for_local_flush(RetainConflictInfoData *data);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *data,
+ TimestampTz now);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3677,6 +3733,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * during change application to prevent it from
+ * remaining unchanged for long periods when the worker
+ * is busy.
+ */
+ if (can_advance_nonremovable_xid(&data, last_recv_timestamp))
+ maybe_advance_nonremovable_xid(&data);
}
else if (c == 'k')
{
@@ -3692,8 +3757,26 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&data);
+
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);
+
+ if (XLogRecPtrIsInvalid(data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&data);
+
+ UpdateWorkerStats(last_received, data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3789,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&data);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3803,6 +3888,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&data);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3838,7 +3925,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;
@@ -3916,6 +4002,298 @@ 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 if concurrent remote transactions persist) ->
+ * 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)
+{
+ 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);
+ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and network
+ * resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ data->candidate_xid = full_xid;
+ data->last_phase_at = InvalidFullTransactionId;
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * 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);
+
+ pq_sendbyte(request_message, 'S');
+ 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->remote_lsn = InvalidXLogRecPtr;
+ data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS;
+
+ /*
+ * Skip calling maybe_advance_nonremovable_xid() since further actions
+ * cannot proceed until the publisher status is received.
+ */
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainConflictInfoData *data)
+{
+ FullTransactionId remote_full_xid;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(data->remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *data)
+{
+ XLogRecPtr writepos;
+ XLogRecPtr flushpos;
+ bool have_pending_txes;
+
+ Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
+ FullTransactionIdIsValid(data->candidate_xid));
+
+ /*
+ * Issue a warning if there is a detected clock skew between the publisher
+ * and subscriber.
+ *
+ * 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(WARNING,
+ errmsg("non-removable 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* Fetch the latest flush position */
+ get_flush_position(&writepos, &flushpos, &have_pending_txes);
+
+ if (flushpos > last_flushpos)
+ last_flushpos = flushpos;
+
+ /* 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, now 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));
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * Determine if the next round of transaction ID advancement can be attempted.
+ *
+ * TODO: The remote flush location (last_flushpos) is currently not updated
+ * during change application, making it impossible to satisfy the condition of
+ * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
+ * Consider updating the remote flush position in the final phase to enable
+ * advancement during change application.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000);
+}
+
/*
* 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 371eef3ddd..00b6411c7e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,50 @@ 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;
+
+ 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 36610a1c7e..8feed1d064 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2929,6 +2929,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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 2d4c870423..c887c39a0a 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2469,6 +2469,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v13_2-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v13_2-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 1572a677376a927be633d11a0d6b5c1d84ac8d2c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v13 2/5] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 16f26866f2..07d0ca477d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1128,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1156,6 +1168,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();
@@ -1176,14 +1190,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1216,6 +1259,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1243,6 +1315,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 887e38d56e..5cb8286d36 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..8c18264f58 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v13_2-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v13_2-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From 555fc893456e97f7bfd8f4b6fa41117c8b661dbd Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v13 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..41ae3e2a07 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1640,6 +1640,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..ce911a41dc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 3e9dd292cb..60e264530a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,6 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2740,15 +2743,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2759,7 +2758,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2778,19 +2777,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3112,7 +3119,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3122,17 +3129,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3144,7 +3159,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3155,7 +3170,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index cbbe8acd38..b8043f263e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v13_2-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v13_2-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From ff581493da838ffb20c10216fcebe972a8d36354 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 2 Dec 2024 19:03:51 +0800
Subject: [PATCH v13 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 29 ++--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 364 insertions(+), 117 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 59bb833f48..4667e1a3ad 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8043,6 +8043,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index ce911a41dc..13f29ea338 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 07d0ca477d..ecd11e9508 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 60e264530a..37b0adfed6 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,9 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2780,7 +2780,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3118,9 +3119,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3132,7 +3133,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3149,8 +3151,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4081,6 +4083,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4304,7 +4310,8 @@ wait_for_local_flush(RetainConflictInfoData *data)
static bool
can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
{
- return data->phase == RCI_GET_CANDIDATE_XID &&
+ return MySubscription->detectupdatedeleted &&
+ data->phase == RCI_GET_CANDIDATE_XID &&
TimestampDifferenceExceeds(data->candidate_xid_time, now,
wal_receiver_status_interval * 1000);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index add7f16c90..19c50e9ea6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4982,6 +4989,7 @@ getSubscriptions(Archive *fout)
i_suboriginremotelsn = PQfnumber(res, "suboriginremotelsn");
i_subenabled = PQfnumber(res, "subenabled");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
subinfo = pg_malloc(ntups * sizeof(SubscriptionInfo));
@@ -5028,6 +5036,8 @@ getSubscriptions(Archive *fout)
pg_strdup(PQgetvalue(res, i, i_subenabled));
subinfo[i].subfailover =
pg_strdup(PQgetvalue(res, i, i_subfailover));
+ subinfo[i].subdetectupdatedeleted =
+ pg_strdup(PQgetvalue(res, i, i_subdetectupdatedeleted));
/* Decide whether we want to dump it */
selectDumpableObject(&(subinfo[i].dobj), fout);
@@ -5270,6 +5280,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (strcmp(subinfo->subfailover, "t") == 0)
appendPQExpBufferStr(query, ", failover = true");
+ if (strcmp(subinfo->subdetectupdatedeleted, "t") == 0)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 d65f558565..80b992d432 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -678,6 +678,7 @@ typedef struct _SubscriptionInfo
char *suborigin;
char *suboriginremotelsn;
char *subfailover;
+ char *subdetectupdatedeleted;
} SubscriptionInfo;
/*
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 0aa39906a1..01e36acc2c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6643,7 +6643,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)
{
@@ -6711,6 +6711,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index b25f3fea56..7058e16e2e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -98,6 +98,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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -151,6 +154,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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v13_2-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v13_2-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 9c77276c5713bebe34ccc6b36490ea30e989ed3f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v13 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
On Monday, December 2, 2024 7:47 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Monday, December 2, 2024 7:34 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V13 patch set which addressed this and pending comments in
[1][2].I am sorry that I missed to merge some changes in previous patch set. Please
check the attached updated patch set V13_2.
Rebased the patch due to the recent commit 08691ea9.
Also, I removed a change in wait_for_local_flush() which was mis-added in V13_2
patch.
Best Regards,
Hou zj
Attachments:
v14-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v14-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 3f460573f2026d20726e2bf79bbc25ead729522b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v14] 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 defined by wal_receiver_status_interval
to minimize performance impact.
This approach ensures that dead tuples are not removed until all concurrent
transactions have been applied. It works for both bidirectional and
non-bidirectional replication scenarios.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 370 ++++++++++++++++++++-
src/backend/replication/walsender.c | 50 +++
src/backend/storage/ipc/procarray.c | 59 ++++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 598 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..eadfcaab09 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1eccb78ddc..d48ddc83af 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 e5fdca8bbf..16f26866f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -450,6 +450,7 @@ retry:
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 925dff9cc4..9cc529536f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,45 @@ 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 candidate_xid_time; /* when the candidate_xid is decided */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ TimestampTz next_attempt_time; /* when to attemp to advance the xid during
+ * change application */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +380,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +426,14 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data);
+static void get_candidate_xid(RetainConflictInfoData *data);
+static void request_publisher_status(RetainConflictInfoData *data);
+static void wait_for_publisher_status(RetainConflictInfoData *data);
+static void wait_for_local_flush(RetainConflictInfoData *data);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *data,
+ TimestampTz now);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3677,6 +3733,15 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * during change application to prevent it from
+ * remaining unchanged for long periods when the worker
+ * is busy.
+ */
+ if (can_advance_nonremovable_xid(&data, last_recv_timestamp))
+ maybe_advance_nonremovable_xid(&data);
}
else if (c == 'k')
{
@@ -3692,8 +3757,26 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&data);
+
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);
+
+ if (XLogRecPtrIsInvalid(data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&data);
+
+ UpdateWorkerStats(last_received, data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3706,6 +3789,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ maybe_advance_nonremovable_xid(&data);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3803,6 +3888,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&data);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3838,7 +3925,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;
@@ -3916,6 +4002,288 @@ 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 if concurrent remote transactions persist) ->
+ * 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)
+{
+ 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);
+ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ now = GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and network
+ * resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ data->candidate_xid = full_xid;
+ data->last_phase_at = InvalidFullTransactionId;
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * 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);
+
+ pq_sendbyte(request_message, 'S');
+ 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->remote_lsn = InvalidXLogRecPtr;
+ data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS;
+
+ /*
+ * Skip calling maybe_advance_nonremovable_xid() since further actions
+ * cannot proceed until the publisher status is received.
+ */
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainConflictInfoData *data)
+{
+ FullTransactionId remote_full_xid;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the remote WAL
+ * position.
+ */
+ if (XLogRecPtrIsInvalid(data->remote_lsn))
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * 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));
+
+ /*
+ * Issue a warning if there is a detected clock skew between the publisher
+ * and subscriber.
+ *
+ * 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(WARNING,
+ errmsg("non-removable 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /* 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, now 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));
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+
+ maybe_advance_nonremovable_xid(data);
+}
+
+/*
+ * Determine if the next round of transaction ID advancement can be attempted.
+ *
+ * TODO: The remote flush location (last_flushpos) is currently not updated
+ * during change application, making it impossible to satisfy the condition of
+ * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
+ * Consider updating the remote flush position in the final phase to enable
+ * advancement during change application.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000);
+}
+
/*
* 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 371eef3ddd..00b6411c7e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,50 @@ 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;
+
+ 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 36610a1c7e..8feed1d064 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2929,6 +2929,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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 2d4c870423..c887c39a0a 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2469,6 +2469,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v14-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v14-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 9e314b4f3218ccbbb1e3b2023ff8218effe42934 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v14 2/5] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 16f26866f2..07d0ca477d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1128,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1156,6 +1168,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();
@@ -1176,14 +1190,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1216,6 +1259,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1243,6 +1315,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 887e38d56e..5cb8286d36 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..8c18264f58 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v14-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v14-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From 7da5e84a82e463c16306760d46cada8677a946d1 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v14 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..41ae3e2a07 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1640,6 +1640,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 54025c9f15..ce911a41dc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 db08a55fd7..f2fad9b5d8 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,6 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2740,15 +2743,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2759,7 +2758,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2778,19 +2777,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3112,7 +3119,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3122,17 +3129,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3144,7 +3159,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3155,7 +3170,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 9575524007..91470bac84 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v14-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v14-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From 9ba4dbc9e4a62698bcfa61f58219caca4caf5c36 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 3 Dec 2024 11:22:56 +0800
Subject: [PATCH v14 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 29 ++--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 364 insertions(+), 117 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bf3cee08a9..6e2aa8a9a0 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8052,6 +8052,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index ce911a41dc..13f29ea338 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 07d0ca477d..ecd11e9508 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f2fad9b5d8..d6629f9803 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2724,9 +2724,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2780,7 +2780,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3118,9 +3119,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3132,7 +3133,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3149,8 +3151,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4081,6 +4083,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4298,7 +4304,8 @@ wait_for_local_flush(RetainConflictInfoData *data)
static bool
can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
{
- return data->phase == RCI_GET_CANDIDATE_XID &&
+ return MySubscription->detectupdatedeleted &&
+ data->phase == RCI_GET_CANDIDATE_XID &&
TimestampDifferenceExceeds(data->candidate_xid_time, now,
wal_receiver_status_interval * 1000);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index ec0cdf4ed7..9bcae370a6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5009,6 +5017,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].subdetectupdatedeleted =
+ (strcmp(PQgetvalue(res, i, i_subdetectupdatedeleted), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5267,6 +5277,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subdetectupdatedeleted)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 2e55a0e3bb..55e32cdff5 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -672,6 +672,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subdetectupdatedeleted;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 2657abdc72..2f8b911574 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index beaff6578a..a811b4a7d1 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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v14-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v14-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 8647a4f8ecfec328a04a272200fcfc5c9a9e78ab Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v14 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
On Tue, Dec 3, 2024 at 9:17 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Also, I removed a change in wait_for_local_flush() which was mis-added in V13_2
patch.
1.
+ if (can_advance_nonremovable_xid(&data, last_recv_timestamp))
+ maybe_advance_nonremovable_xid(&data);
In can_advance_nonremovable_xid(), we determine whether to advance the
oldest xid based on 'last_recv_timestamp' and then again in
maybe_advance_nonremovable_xid()->get_candidate_xid(), we compare it
with the current time. How does that make sense? Shall we use
'last_recv_timestamp' directly in get_candidate_xid() as that will
avoid the additional time check in can_advance_nonremovable_xid()?
2.
+ TimestampTz next_attempt_time; /* when to attemp to advance the xid during
+ * change application */
+} RetainConflictInfoData;
This new variable introduced in this version is not used in the patch.
Any reason or just a leftover?
Apart from the above, I have made a few updates in the comments in the
attached. Please include those after review.
--
With Regards,
Amit Kapila.
Attachments:
v14_amit_1.patch.txttext/plain; charset=US-ASCII; name=v14_amit_1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 97f4d9fcba..06ba6d3a64 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3736,9 +3736,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/*
* Attempt to advance the non-removable transaction ID
- * during change application to prevent it from
- * remaining unchanged for long periods when the worker
- * is busy.
+ * to avoid accumulating dead rows when the worker is
+ * busy.
*/
if (can_advance_nonremovable_xid(&data, last_recv_timestamp))
maybe_advance_nonremovable_xid(&data);
@@ -3770,6 +3769,12 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
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");
@@ -4164,8 +4169,8 @@ request_publisher_status(RetainConflictInfoData *data)
data->phase = RCI_WAIT_FOR_PUBLISHER_STATUS;
/*
- * Skip calling maybe_advance_nonremovable_xid() since further actions
- * cannot proceed until the publisher status is received.
+ * Skip calling maybe_advance_nonremovable_xid() since further transition
+ * is possible only once we receive the publisher status message.
*/
}
@@ -4244,7 +4249,14 @@ wait_for_local_flush(RetainConflictInfoData *data)
if (!AllTablesyncsReady())
return;
- /* Return to wait for the changes to be applied */
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
if (last_flushpos < data->remote_lsn)
return;
@@ -4268,7 +4280,7 @@ wait_for_local_flush(RetainConflictInfoData *data)
}
/*
- * Determine if the next round of transaction ID advancement can be attempted.
+ * Determine if we can attempt to advance transaction ID.
*
* TODO: The remote flush location (last_flushpos) is currently not updated
* during change application, making it impossible to satisfy the condition of
On Wed, Dec 4, 2024 at 4:29 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
1. + if (can_advance_nonremovable_xid(&data, last_recv_timestamp)) + maybe_advance_nonremovable_xid(&data);In can_advance_nonremovable_xid(), we determine whether to advance the
oldest xid based on 'last_recv_timestamp' and then again in
maybe_advance_nonremovable_xid()->get_candidate_xid(), we compare it
with the current time. How does that make sense? Shall we use
'last_recv_timestamp' directly in get_candidate_xid() as that will
avoid the additional time check in can_advance_nonremovable_xid()?2. + TimestampTz next_attempt_time; /* when to attemp to advance the xid during + * change application */ +} RetainConflictInfoData;This new variable introduced in this version is not used in the patch.
Any reason or just a leftover?Apart from the above, I have made a few updates in the comments in the
attached. Please include those after review.
A few more comments:
1.
+static void
+wait_for_local_flush(RetainConflictInfoData *data)
{
...
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+
+ maybe_advance_nonremovable_xid(data);
+}
Isn't it better to reset all the fields of data before the next round
of GET_CANDIDATE_XID phase? If we do that then we don't need to reset
data->remote_lsn = InvalidXLogRecPtr; and data->last_phase_at =
InvalidFullTransactionId; individually in request_publisher_status()
and get_candidate_xid() respectively. Also, it looks clean and logical
to me unless I am missing something.
2.
+ /*
+ * Issue a warning if there is a detected clock skew between the publisher
+ * and subscriber.
+ *
+ * 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(WARNING,
+ errmsg("non-removable transaction ID may be advanced prematurely"),
+ errdetail("The clock on the publisher is behind that of the subscriber."));
Shouldn't this be an ERROR as this will lead to the removal of rows
required to detect update_delete conflict?
Apart from the above, I have made a few more updates in the comments
in the attached. Please include those after review.
--
With Regards,
Amit Kapila.
Attachments:
v14_amit_2.patch.txttext/plain; charset=US-ASCII; name=v14_amit_2.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 06ba6d3a64..e89e811c51 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4135,6 +4135,7 @@ get_candidate_xid(RetainConflictInfoData *data)
data->last_phase_at = InvalidFullTransactionId;
data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+ /* process the next phase */
maybe_advance_nonremovable_xid(data);
}
@@ -4156,6 +4157,10 @@ request_publisher_status(RetainConflictInfoData *data)
else
resetStringInfo(request_message);
+ /*
+ * We send the current time to update the remote walsender's latest reply
+ * message received time.
+ */
pq_sendbyte(request_message, 'S');
pq_sendint64(request_message, GetCurrentTimestamp());
@@ -4213,6 +4218,7 @@ wait_for_publisher_status(RetainConflictInfoData *data)
else
data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+ /* process the next phase */
maybe_advance_nonremovable_xid(data);
}
@@ -4226,8 +4232,10 @@ wait_for_local_flush(RetainConflictInfoData *data)
FullTransactionIdIsValid(data->candidate_xid));
/*
- * Issue a warning if there is a detected clock skew between the publisher
- * and subscriber.
+ * 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.
@@ -4235,7 +4243,7 @@ wait_for_local_flush(RetainConflictInfoData *data)
if (TimestampDifferenceExceeds(data->reply_time,
data->candidate_xid_time, 0))
ereport(WARNING,
- errmsg("non-removable transaction ID may be advanced prematurely"),
+ errmsg("oldest_nonremovable_xid transaction ID may be advanced prematurely"),
errdetail("The clock on the publisher is behind that of the subscriber."));
/*
@@ -4276,6 +4284,7 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->phase = RCI_GET_CANDIDATE_XID;
+ /* process the next phase */
maybe_advance_nonremovable_xid(data);
}
On Thursday, December 5, 2024 6:00 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 4, 2024 at 4:29 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:1. + if (can_advance_nonremovable_xid(&data, last_recv_timestamp)) + maybe_advance_nonremovable_xid(&data);In can_advance_nonremovable_xid(), we determine whether to advance the
oldest xid based on 'last_recv_timestamp' and then again in
maybe_advance_nonremovable_xid()->get_candidate_xid(), we compare it
with the current time. How does that make sense? Shall we use
'last_recv_timestamp' directly in get_candidate_xid() as that will
avoid the additional time check in can_advance_nonremovable_xid()?
Agreed.
I added a field to RetainConflictInfoData to store the last_recv_timestamp and
am using it directly in get_candidate_xid(). If no message has been received,
we can still obtain the current timestamp.
2. + TimestampTz next_attempt_time; /* when to attemp to advance the xid +during + * change application */ +} RetainConflictInfoData;This new variable introduced in this version is not used in the patch.
Any reason or just a leftover?
It was a leftover, removed.
Apart from the above, I have made a few updates in the comments in the
attached. Please include those after review.A few more comments: 1. +static void +wait_for_local_flush(RetainConflictInfoData *data) { ... + + data->phase = RCI_GET_CANDIDATE_XID; + + maybe_advance_nonremovable_xid(data); +}Isn't it better to reset all the fields of data before the next round of
GET_CANDIDATE_XID phase? If we do that then we don't need to reset
data->remote_lsn = InvalidXLogRecPtr; and data->last_phase_at =
InvalidFullTransactionId; individually in request_publisher_status() and
get_candidate_xid() respectively. Also, it looks clean and logical to me unless I
am missing something.
The remote_lsn was used to determine whether a status is received, so was reset
each time in request_publisher_status. To make it more straightforward, I added
a new function parameter 'status_received', which would be set to true when
calling maybe_advance_nonremovable_xid() on receving the status. After this
change, there is no need to reset the remote_lsn.
2. + /* + * Issue a warning if there is a detected clock skew between the + publisher + * and subscriber. + * + * 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(WARNING, + errmsg("non-removable transaction ID may be advanced prematurely"), + errdetail("The clock on the publisher is behind that of the + subscriber."));Shouldn't this be an ERROR as this will lead to the removal of rows required to
detect update_delete conflict?
Agreed. Changed to ERROR.
Apart from the above, I have made a few more updates in the comments in the
attached. Please include those after review.
Thanks, I have checked and merged.
Here is V15 patch set which addressed above comments. I also fixed a bug that
the table sync worker mis-sent the request to publisher.
Best Regards,
Hou zj
Attachments:
v15-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v15-0005-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From cb25aa48cc96e1c418a19348f83b644fe239bff8 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Sun, 1 Sep 2024 17:50:31 +0800
Subject: [PATCH v15 5/5] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/034_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/034_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index c591cd7d61..6303322d41 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -40,6 +40,7 @@ tests += {
't/031_column_list.pl',
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
+ 't/034_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/034_confl_update_deleted.pl b/src/test/subscription/t/034_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/034_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v15-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v15-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 5771bdfcce839f8c6e7a461eefa87bc4e38e8bda Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v15] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 405 ++++++++++++++++++++-
src/backend/replication/walsender.c | 50 +++
src/backend/storage/ipc/procarray.c | 59 +++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 633 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..eadfcaab09 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1eccb78ddc..d48ddc83af 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 e5fdca8bbf..16f26866f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -450,6 +450,7 @@ retry:
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 46d3ad566f..a75a3691dc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,44 @@ 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 candidate_xid_time; /* when the candidate_xid is decided */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ TimestampTz last_recv_time; /* when the last message was received */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +379,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +425,15 @@ 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 bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3651,6 +3707,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);
@@ -3677,6 +3735,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3758,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);
@@ -3706,6 +3796,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)
{
/*
@@ -3803,6 +3898,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
@@ -3838,7 +3935,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;
@@ -3916,6 +4012,313 @@ 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 if concurrent remote transactions persist) ->
+ * 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)
+{
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and
+ * network resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ data->last_phase_at = InvalidFullTransactionId;
+ 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, 'S');
+ 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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, now 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));
+
+ data->phase = RCI_GET_CANDIDATE_XID;
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ *
+ * TODO: The remote flush location (last_flushpos) is currently not updated
+ * during change application, making it impossible to satisfy the condition of
+ * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
+ * Consider updating the remote flush position in the final phase to enable
+ * advancement during change application.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
/*
* 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 371eef3ddd..00b6411c7e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,50 @@ 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;
+
+ 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 36610a1c7e..8feed1d064 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2929,6 +2929,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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 ce33e55bf1..14bef051b6 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2470,6 +2470,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v15-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v15-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 7edd69f7160e801a9ddb65f662f1b26c323c1b31 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v15 2/5] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 16f26866f2..07d0ca477d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1128,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1156,6 +1168,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();
@@ -1176,14 +1190,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1216,6 +1259,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1243,6 +1315,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 4a206f9527..937afa5b6a 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..8c18264f58 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v15-0003-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v15-0003-Support-the-conflict-detection-for-update_delete.patchDownload
From 53fa776e67f5c4f8a89997c338f5963545ae0033 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Sep 2024 10:52:35 +0800
Subject: [PATCH v15 3/5] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 6 +-
src/include/replication/conflict.h | 3 +
src/test/regress/expected/rules.out | 3 +-
11 files changed, 239 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..41ae3e2a07 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1640,6 +1640,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index cfdf2eedf4..89c177671b 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -14,11 +14,13 @@
#include "postgres.h"
+#include "access/commit_ts.h"
#include "access/genam.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"
@@ -26,6 +28,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"
@@ -35,7 +38,7 @@
static bool tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq);
+ TypeCacheEntry **eq, Bitmapset *columns);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -264,7 +267,7 @@ retry:
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;
}
@@ -323,7 +326,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -348,6 +351,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -423,7 +432,7 @@ retry:
/* 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;
@@ -474,6 +483,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 3ce622c36f..1ca4ee7efa 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2725,6 +2725,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2741,15 +2744,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2760,7 +2759,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2779,19 +2778,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3113,7 +3120,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3123,17 +3130,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3145,7 +3160,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3156,7 +3171,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 9575524007..91470bac84 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 69c3ebff00..0cf6af55ed 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -665,7 +665,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v15-0004-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v15-0004-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From c88c3d9e682a91168206710de716d0561afa8522 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 6 Dec 2024 14:07:00 +0800
Subject: [PATCH v15 4/5] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 362 insertions(+), 116 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bf3cee08a9..6e2aa8a9a0 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8052,6 +8052,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 89c177671b..9aae236c57 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -546,14 +546,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 07d0ca477d..ecd11e9508 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 1ca4ee7efa..010b7cbd1b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2725,9 +2725,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2781,7 +2781,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3119,9 +3120,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3133,7 +3134,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3150,8 +3152,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4093,6 +4095,10 @@ static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* The non-removable transaction ID for a subscription is centrally
* managed by the main apply worker.
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index ec0cdf4ed7..9bcae370a6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5009,6 +5017,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].subdetectupdatedeleted =
+ (strcmp(PQgetvalue(res, i, i_subdetectupdatedeleted), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5267,6 +5277,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subdetectupdatedeleted)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 2e55a0e3bb..55e32cdff5 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -672,6 +672,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subdetectupdatedeleted;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 2657abdc72..2f8b911574 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index beaff6578a..a811b4a7d1 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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
Here is a summary of tests targeted to the Publisher node in a
Publisher-Subscriber setup.
(All tests done with v14 patch-set)
----------------------------
Performance Tests:
----------------------------
Test machine details:
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :120 - 800GB RAM
Setup:
- Created two nodes ( 'Pub' and 'Sub'), with logical replication.
- Configurations for Both Nodes:
shared_buffers = 40GB
max_worker_processes = 32
max_parallel_maintenance_workers = 24
max_parallel_workers = 32
checkpoint_timeout = 1d
max_wal_size = 24GB
min_wal_size = 15GB
autovacuum = off
- Additional setting on Sub: 'track_commit_timestamp = on' (required
for the feature).
- Initial data insertion via 'pgbench' with scale factor 100 on both nodes.
Workload:
- Ran pgbench with 60 clients for the publisher.
- The duration was 120s, and the measurement was repeated 10 times.
(Attached the scripts - "measure_pub.sh" and "setup_pub.sh" to
automate the above steps.)
Test Scenarios & Results:
- TPS (Transactions Per Second) were measured for the following scenarios:
- pgHead (with default wal_receiver_status_interval): Median TPS =
57708.24016
- pgHead (with wal_receiver_status_interval = 1s): Median TPS = 57467.58763
- pgHead + patch (detect_update_deleted=ON and default
wal_receiver_status_interval): Median TPS = 58598.98634
- pgHead + patch (detect_update_deleted=ON and
wal_receiver_status_interval = 1s): Median TPS = 58549.6975
Observation:
- No performance regression observed with the patch applied.
- 1-2% TPS improvement observed with the patch, likely due to some
compiler-level optimizations with the patch.
- The results were consistent across runs.
Detailed Results Table:
-- each cell shows the TPS in each case.
-- patch(ON) means patched and detect_update_deleted=ON is set.
-- interval=1s means wal_receiver_status_interval = 1s
Run# | pgHEAD | pgHead(interval=1s) | pgHead+patch(ON) |
pgHead+patch(ON)(interval=1s)
1 | 57737.49728 | 57547.05839 | 58656.83886 | 58570.13829
2 | 57650.25327 | 57795.05524 | 58397.82185 | 58624.47823
3 | 57708.24016 | 57467.58763 | 58618.85799 | 58381.69373
4 | 57677.51559 | 57438.90709 | 58640.20102 | 58529.25671
5 | 57845.84897 | 57390.75618 | 58616.45811 | 58122.4126
6 | 57667.91187 | 57598.73271 | 58567.73797 | 58576.74363
7 | 57821.66669 | 57642.50589 | 58573.27994 | 58507.67747
8 | 57649.50195 | 57792.58659 | 58581.51457 | 58605.13375
9 | 57672.72002 | 57656.59962 | 58677.14569 | 58512.77625
10 | 57606.99544 | 57660.21762 | 58502.80998 | 58590.33837
Median | 57708.24016 | 57467.58763 | 58598.98634 | 58549.6975
std. dev. | 76.9972099 | 138.9779921 | 82.35632411 | 150.2302062
~~~~
----------------------------
Walsender Profiling:
----------------------------
- Profiling was done for the 'walsender' process with the patch
enabled (both default and 'wal_receiver_status_interval=1s' cases).
- Functions GetOldestTransactionIdInCommit() and
ReadNextFullTransactionId() do not appear in the 'walsender' profile
in both cases.
- The walsender profile reports are attached.
~~~~
Note: Some performance regression was observed on the Subscriber side
with the patch and detect_update_deleted=ON. To detect update_deleted
conflicts, dead tuples are retained for a bit longer period, which is
likely the cause of the observed TPS regression.
A detailed analysis of the Subscriber-side cases will be shared in a
separate email.
--
Thanks,
Nisha
On Mon, Dec 9, 2024 at 3:20 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
Here is a summary of tests targeted to the Publisher node in a
Publisher-Subscriber setup.
(All tests done with v14 patch-set)----------------------------
Performance Tests:
----------------------------
Test machine details:
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :120 - 800GB RAMSetup:
- Created two nodes ( 'Pub' and 'Sub'), with logical replication.
- Configurations for Both Nodes:shared_buffers = 40GB
max_worker_processes = 32
max_parallel_maintenance_workers = 24
max_parallel_workers = 32
checkpoint_timeout = 1d
max_wal_size = 24GB
min_wal_size = 15GB
autovacuum = off- Additional setting on Sub: 'track_commit_timestamp = on' (required
for the feature).
- Initial data insertion via 'pgbench' with scale factor 100 on both nodes.Workload:
- Ran pgbench with 60 clients for the publisher.
- The duration was 120s, and the measurement was repeated 10 times.
You didn't mention it is READONLY or READWRITE tests but I think it is
later. I feel it is better to run these tests for 15 minutes, repeat
them 3 times, and get the median data for those. Also, try to run it
for lower client counts like 2, 16, 32. Overall, the conclusion may be
same but it will rule out the possibility of any anomaly.
With Regards,
Amit Kapila.
Here are the test steps and analysis for epoch-related handling
(Tested with v15 Patch-Set).
In the 'update_deleted' detection design, the launcher process
compares XIDs to track minimum XIDs, and the apply workers maintain
the oldest running XIDs. The launcher also requests publisher status
at regular intervals which also includes the epoch info. So, proper
epoch handling is required for the smooth functioning during XID
wraparound.
Functions requiring epoch handling:
1) 'get_candidate_xid()': Tracks the node's oldest running XID and
identifies the next candidate XID to advance the oldest non-removable
XID of an apply worker.
2) 'wait_for_publisher_status()': Tracks the publisher’s oldest and
next XIDs for monitoring concurrent remote transactions.
-- To test the epoch handling, I added extra LOG statements in above
functions and the launcher code. The patches for these changes are
attached (applies atop v15-0005).
The tests confirmed that epoch handling works correctly during XID
wraparound on both the publisher and subscriber sides. Detailed test
steps and results are provided below.
~~~~
Setup:
- Created two nodes, 'Pub' and 'Sub', with logical replication.
- On both nodes, configured 'autovacuum_naptime = 1s' to allow
frequent vacuuming while consuming XIDs rapidly.
- On Sub, created a subscription for a table subscribed to all changes
from Pub.
- Installed and enabled the 'xid_wraparound' extension on both nodes:
CREATE EXTENSION xid_wraparound;
~~~~
-----------------------------------------------------------------
Case-1: When XID Wraparound Happens on Sub
-----------------------------------------------------------------
Scenario:
In 'get_candidate_xid()', 'oldest_running_xid' and 'next_full_xid'
have different epochs, meaning, an old epoch transaction is running,
and a xid-wraparound happens on the subscriber.
Test Steps:
Perform below steps at Sub node:
1. Consume 4 Billion XIDs in Batches (400M each).
-- script "consume_4B_xids.sh" is attached which is used to consume the xids.
2. Set 'detect_update_deleted=ON' for the subscription.
3. Hold a transaction with an old XID before Wraparound:
-- Start a new session, begin a transaction, and leave it open. This
transaction will have an XID close to 4 billion.
4. In another session, trigger wraparound by consuming remaining XIDs
(2^32 - 4B):
SELECT consume_xids('294966530');
-- At Sub, the newly added log will show that wraparound happened and
epoch was handled correctly by choosing the correct
candidate_full_xid.
LOG: XXX: oldest_running_xid = 4000000762
LOG: XXX: next_full_xid = 766
LOG: XXX: xid WRAPAROUND happened!!!
LOG: XXX: candidate full_xid = 4000000762
5. Confirm launcher updates "pg_conflict_detection" slot's xmin with new epoch:
- End the open transaction.
- Verify that the oldest running xid is now updated to the new epoch xid
LOG: XXX: oldest_running_xid = 766
LOG: XXX: next_full_xid = 766
LOG: XXX: candidate full_xid = 766
- Confirm the launcher updates the new epoch xid as xmin:
LOG: XXX: launcher new_xid = 766
LOG: XXX: launcher current slot xmin = 4000000762
LOG: XXX: launcher full_xmin = 4000000762
LOG: XXX: launcher updated xmin = 766
postgres=# SELECT slot_name, slot_type, active, xmin FROM
pg_replication_slots;
slot_name | slot_type | active | xmin
-----------------------+-----------+--------+------
pg_conflict_detection | physical | t | 766
(1 row)
~~~~
-----------------------------------------------------------------
Case-2: When XID Wraparound Happens on Pub
-----------------------------------------------------------------
Scenario:
In 'wait_for_publisher_status()', 'data->last_phase_at' (oldest
commiting remote XID) and 'remote_next_xid' have different epochs,
meaning, an old epoch transaction is in commit phase on remote(Pub),
and wraparound happens on the publisher node.
Test Steps:
1. Consume 4 Billion XIDs in Batches (400M each) on the Publisher node.
-- script "consume_4B_xids.sh" is attached which is used to consume the xids.
2. At sub, set 'detect_update_deleted=ON' for the subscription.
3. Confirm the latest remote XID are updated on Sub:
LOG: XXX: last_phase_at = 4000000796
LOG: XXX: remote_oldestxid = 4000000796
LOG: XXX: remote_nextxid = 4000000796
LOG: XXX: remote_full_xid = 4000000796
LOG: XXX: remote concurrent txn completed
4. Hold a transaction in the commit phase:
- Attach a debugger to a session, start a transaction, and hold it
at 'XactLogCommitRecord()'.
- This step is required because the launcher at sub tracks remote
concurrent transactions which are currently committing.
5. In another session, trigger wraparound by consuming remaining XIDs
(2^32 - 4B):
SELECT consume_xids('294966530');
-- At sub, the logs confirm that wraparound happened on Pub node:
LOG: XXX: last_phase_at = 4000000797
LOG: XXX: remote_oldestxid = 4000000796
LOG: XXX: remote_nextxid = 801
LOG: XXX: xid WRAPAROUND happened on Publisher!!!
LOG: XXX: remote_full_xid = 4000000796
6. Release debugger and confirm that remote's oldest XID updated to
the new epoch:
LOG: XXX: last_phase_at = 4000000797
LOG: XXX: remote_oldestxid = 801
LOG: XXX: remote_nextxid = 801
LOG: XXX: remote_full_xid = 801
LOG: XXX: remote concurrent txn completed
----------------------------- END -------------------------------
Thanks,
Nisha
Attachments:
epoch_test_case-1_logs.patchapplication/octet-stream; name=epoch_test_case-1_logs.patchDownload
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ecd11e9508..9ab91c3197 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1392,14 +1392,22 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
* xmin of the replication slot. This handles the case where transaction
* ID wraparound has occurred.
*/
+elog(LOG,"XXX: launcher new_xid= %u",next_xid);
+elog(LOG,"XXX: launcher current slot xmin = %u",MyReplicationSlot->data.xmin);
+
if (next_xid < MyReplicationSlot->data.xmin)
xmin_epoch--;
full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
MyReplicationSlot->data.xmin);
+elog(LOG,"XXX: launcher full_xmin= %u",XidFromFullTransactionId(full_xmin));
if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ {
+ elog(LOG,"XXX: launcher returned without updating xmin!");
return false;
+ }
+
SpinLockAcquire(&MyReplicationSlot->mutex);
MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
@@ -1411,6 +1419,8 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
ReplicationSlotSave();
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+elog(LOG,"XXX: launcher 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
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index afd31e1525..3d1024b96e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4136,12 +4136,20 @@ get_candidate_xid(RetainConflictInfoData *data)
next_full_xid = ReadNextFullTransactionId();
epoch = EpochFromFullTransactionId(next_full_xid);
+elog(LOG,"XXX: oldest_running_xid = %u",oldest_running_xid);
+elog(LOG,"XXX: next_full_xid = %u ",XidFromFullTransactionId(next_full_xid));
+
/* Compute the epoch of the oldest_running_xid */
if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ {
+ elog(LOG,"XXX: xid WRAPAROUND happend!!!");
epoch--;
+ }
full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+elog(LOG,"XXX: candidate full_xid = %u ",XidFromFullTransactionId(full_xid));
+
/* Return if the oldest_nonremovable_xid cannot be advanced */
if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
full_xid))
epoch_test_case-2_logs.patchapplication/octet-stream; name=epoch_test_case-2_logs.patchDownload
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ecd11e9508..9ab91c3197 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1392,14 +1392,22 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
* xmin of the replication slot. This handles the case where transaction
* ID wraparound has occurred.
*/
+elog(LOG,"XXX: launcher new_xid= %u",next_xid);
+elog(LOG,"XXX: launcher current slot xmin = %u",MyReplicationSlot->data.xmin);
+
if (next_xid < MyReplicationSlot->data.xmin)
xmin_epoch--;
full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
MyReplicationSlot->data.xmin);
+elog(LOG,"XXX: launcher full_xmin= %u",XidFromFullTransactionId(full_xmin));
if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ {
+ elog(LOG,"XXX: launcher returned without updating xmin!");
return false;
+ }
+
SpinLockAcquire(&MyReplicationSlot->mutex);
MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
@@ -1411,6 +1419,8 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
ReplicationSlotSave();
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+elog(LOG,"XXX: launcher 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
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index afd31e1525..d0d3b38433 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4209,13 +4209,20 @@ wait_for_publisher_status(RetainConflictInfoData *data)
if (!FullTransactionIdIsValid(data->last_phase_at))
data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
data->remote_nextxid);
-
+elog(LOG,"XXX: last_phase_at = %u",XidFromFullTransactionId(data->last_phase_at));
+elog(LOG,"XXX: remote_oldestxid = %u",data->remote_oldestxid);
+elog(LOG,"XXX: remote_nextxid = %u",data->remote_nextxid);
/* Compute the epoch of the remote oldest running transaction ID */
if (data->remote_oldestxid > data->remote_nextxid)
+ {
+ elog(LOG,"XXX: xid WRAPAROUND happend on Publisher!!!");
remote_epoch--;
+ }
+
remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
data->remote_oldestxid);
+elog(LOG,"XXX: remote_full_xid = %u",XidFromFullTransactionId(remote_full_xid));
/*
* Check if all remote concurrent transactions that were active at the
@@ -4225,7 +4232,10 @@ wait_for_publisher_status(RetainConflictInfoData *data)
*/
if (FullTransactionIdPrecedesOrEquals(data->last_phase_at,
remote_full_xid))
+ {
data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ elog(LOG,"XXX: remote concurrent txn completed");
+ }
else
data->phase = RCI_REQUEST_PUBLISHER_STATUS;
On Fri, Dec 6, 2024 at 1:28 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thursday, December 5, 2024 6:00 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
A few more comments: 1. +static void +wait_for_local_flush(RetainConflictInfoData *data) { ... + + data->phase = RCI_GET_CANDIDATE_XID; + + maybe_advance_nonremovable_xid(data); +}Isn't it better to reset all the fields of data before the next round of
GET_CANDIDATE_XID phase? If we do that then we don't need to reset
data->remote_lsn = InvalidXLogRecPtr; and data->last_phase_at =
InvalidFullTransactionId; individually in request_publisher_status() and
get_candidate_xid() respectively. Also, it looks clean and logical to me unless I
am missing something.The remote_lsn was used to determine whether a status is received, so was reset
each time in request_publisher_status. To make it more straightforward, I added
a new function parameter 'status_received', which would be set to true when
calling maybe_advance_nonremovable_xid() on receving the status. After this
change, there is no need to reset the remote_lsn.
As part of the above comment, I had asked for three things (a) avoid
setting data->remote_lsn = InvalidXLogRecPtr; in
request_publisher_status(); (b) avoid setting data->last_phase_at
=InvalidFullTransactionId; in get_candidate_xid(); (c) reset data in
wait_for_local_flush() after wait is over. You only did (a) in the
patch and didn't mention anything about (b) or (c). Is that
intentional? If so, what is the reason?
*
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+
Isn't it better to make this an inline function as it contains just one check?
*
+ /*
+ * The non-removable transaction ID for a subscription is centrally
+ * managed by the main apply worker.
+ */
+ if (!am_leader_apply_worker())
I have tried to improve this comment in the attached.
--
With Regards,
Amit Kapila.
Attachments:
v15_amit_1.patch.txttext/plain; charset=US-ASCII; name=v15_amit_1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index a75a3691dc..b1b77e4a1e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4078,8 +4078,9 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received)
{
/*
- * The non-removable transaction ID for a subscription is centrally
- * managed by the main apply worker.
+ * 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;
On Wednesday, December 11, 2024 1:06 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Dec 6, 2024 at 1:28 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Thursday, December 5, 2024 6:00 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:
A few more comments: 1. +static void +wait_for_local_flush(RetainConflictInfoData *data) { ... + + data->phase = RCI_GET_CANDIDATE_XID; + + maybe_advance_nonremovable_xid(data); +}Isn't it better to reset all the fields of data before the next
round of GET_CANDIDATE_XID phase? If we do that then we don't need
to reset
data->remote_lsn = InvalidXLogRecPtr; and data->last_phase_at =
InvalidFullTransactionId; individually in request_publisher_status()
and
get_candidate_xid() respectively. Also, it looks clean and logical
to me unless I am missing something.The remote_lsn was used to determine whether a status is received, so
was reset each time in request_publisher_status. To make it more
straightforward, I added a new function parameter 'status_received',
which would be set to true when calling
maybe_advance_nonremovable_xid() on receving the status. After thischange, there is no need to reset the remote_lsn.
As part of the above comment, I had asked for three things (a) avoid setting
data->remote_lsn = InvalidXLogRecPtr; in request_publisher_status(); (b)
avoid setting data->last_phase_at =InvalidFullTransactionId; in
get_candidate_xid(); (c) reset data in
wait_for_local_flush() after wait is over. You only did (a) in the patch and didn't
mention anything about (b) or (c). Is that intentional? If so, what is the reason?
I think I misunderstood the intention, so will address in next version.
* +static bool +can_advance_nonremovable_xid(RetainConflictInfoData *data) { +Isn't it better to make this an inline function as it contains just one check?
Agreed. Will address in next version.
* + /* + * The non-removable transaction ID for a subscription is centrally + * managed by the main apply worker. + */ + if (!am_leader_apply_worker())I have tried to improve this comment in the attached.
Thanks, will check and merge the next version.
Best Regards,
Hou zj
On Wednesday, December 11, 2024 3:34 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Wednesday, December 11, 2024 1:06 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:On Fri, Dec 6, 2024 at 1:28 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:On Thursday, December 5, 2024 6:00 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:
A few more comments: 1. +static void +wait_for_local_flush(RetainConflictInfoData *data) { ... + + data->phase = RCI_GET_CANDIDATE_XID; + + maybe_advance_nonremovable_xid(data); +}Isn't it better to reset all the fields of data before the next
round of GET_CANDIDATE_XID phase? If we do that then we don't need
to reset
data->remote_lsn = InvalidXLogRecPtr; and data->last_phase_at =
InvalidFullTransactionId; individually in
request_publisher_status() and
get_candidate_xid() respectively. Also, it looks clean and logical
to me unless I am missing something.The remote_lsn was used to determine whether a status is received,
so was reset each time in request_publisher_status. To make it more
straightforward, I added a new function parameter 'status_received',
which would be set to true when calling
maybe_advance_nonremovable_xid() on receving the status. After thischange, there is no need to reset the remote_lsn.
As part of the above comment, I had asked for three things (a) avoid
setting
data->remote_lsn = InvalidXLogRecPtr; in request_publisher_status();
data->(b)
avoid setting data->last_phase_at =InvalidFullTransactionId; in
get_candidate_xid(); (c) reset data in
wait_for_local_flush() after wait is over. You only did (a) in the
patch and didn't mention anything about (b) or (c). Is that intentional? If so,what is the reason?
I think I misunderstood the intention, so will address in next version.
* +static bool +can_advance_nonremovable_xid(RetainConflictInfoData *data) { +Isn't it better to make this an inline function as it contains just one check?
Agreed. Will address in next version.
* + /* + * The non-removable transaction ID for a subscription is centrally + * managed by the main apply worker. + */ + if (!am_leader_apply_worker())I have tried to improve this comment in the attached.
Thanks, will check and merge the next version.
Attach the V16 patch set which addressed above comments.
There is a new 0002 patch where I tried to dynamically adjust the interval for
advancing the transaction ID. Instead of always waiting for
wal_receiver_status_interval, we can start with a short interval and increase
it if there is no activity (no xid assigned on subscriber), but not beyond
wal_receiver_status_interval.
The intention is to more effectively advance xid to avoid retaining too much
dead tuples. My colleague will soon share detailed performance data and
analysis related to this enhancement.
Best Regards,
Hou zj
Attachments:
v16-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v16-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 24bc64567914458363108a2a90a697f2ee2d125c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:51:23 +0800
Subject: [PATCH v16 6/6] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/035_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/035_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index b2395e7b57..a80c7f1656 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_confl_update_deleted.pl b/src/test/subscription/t/035_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/035_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v16-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v16-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From d87b5dbbceb8430f6b77f63f1941300e8053798f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v16 1/2] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 411 ++++++++++++++++++++-
src/backend/replication/walsender.c | 50 +++
src/backend/storage/ipc/procarray.c | 59 +++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 639 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..eadfcaab09 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('S')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 3ebd7c4041..611926c7d2 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 e5fdca8bbf..16f26866f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -450,6 +450,7 @@ retry:
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 46d3ad566f..6b79aa441b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,44 @@ 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 last_recv_time; /* when the last message was received */
+ TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +379,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +425,15 @@ 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 inline bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3651,6 +3707,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);
@@ -3677,6 +3735,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3758,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);
@@ -3706,6 +3796,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)
{
/*
@@ -3803,6 +3898,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
@@ -3838,7 +3935,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;
@@ -3916,6 +4012,319 @@ 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 if concurrent remote transactions persist) ->
+ * 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and
+ * network resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ 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, 'S');
+ 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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, now 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.
+ */
+ memset(data, 0, offsetof(RetainConflictInfoData, candidate_xid_time));
+
+ Assert(data->phase == RCI_GET_CANDIDATE_XID);
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ *
+ * TODO: The remote flush location (last_flushpos) is currently not updated
+ * during change application, making it impossible to satisfy the condition of
+ * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
+ * Consider updating the remote flush position in the final phase to enable
+ * advancement during change application.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
/*
* 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 371eef3ddd..00b6411c7e 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,50 @@ 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;
+
+ 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 c769b1aa3e..ae6da9fd91 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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 ce33e55bf1..14bef051b6 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2470,6 +2470,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v16-0002-Dynamically-adjust-xid-advancement-interval.patchapplication/octet-stream; name=v16-0002-Dynamically-adjust-xid-advancement-interval.patchDownload
From 272f3c3ef468fafd2ee06875d55358525e1df098 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 16:37:00 +0800
Subject: [PATCH v16 2/2] Dynamically adjust xid advancement interval
If no new transaction ID has been assigned since the last advancement, the
interval is doubled. This increase is limited by the
wal_receiver_status_interval if it is not zero, or otherwise restricted to a
maximum of 3 minutes. If a new transaction ID is detected, the interval is
reset to a minimum of 100ms.
---
src/backend/replication/logical/worker.c | 62 +++++++++++++++++++++++-
1 file changed, 61 insertions(+), 1 deletion(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6b79aa441b..9e494351eb 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -313,8 +313,19 @@ typedef struct RetainConflictInfoData
TimestampTz reply_time; /* when the publisher responds with status */
TimestampTz last_recv_time; /* when the last message was received */
TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+ int xid_advancement_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.
+ */
+#define MIN_XID_ADVANCEMENT_INTERVAL 100
+#define MAX_XID_ADVANCEMENT_INTERVAL 180000L
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -428,6 +439,8 @@ 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 adjust_xid_advancement_interval(RetainConflictInfoData *data,
+ bool new_xid_found);
static void request_publisher_status(RetainConflictInfoData *data);
static void wait_for_publisher_status(RetainConflictInfoData *data,
bool status_received);
@@ -3835,6 +3848,13 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to attempt advancing the
+ * non-removable transaction ID.
+ */
+ if (data.phase == RCI_GET_CANDIDATE_XID && data.xid_advancement_interval)
+ wait_time = Min(wait_time, data.xid_advancement_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -4129,7 +4149,7 @@ get_candidate_xid(RetainConflictInfoData *data)
* can consider the other interval or a separate GUC if the need arises.
*/
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
- wal_receiver_status_interval * 1000))
+ data->xid_advancement_interval))
return;
data->candidate_xid_time = now;
@@ -4147,7 +4167,12 @@ get_candidate_xid(RetainConflictInfoData *data)
/* Return if the oldest_nonremovable_xid cannot be advanced */
if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
full_xid))
+ {
+ adjust_xid_advancement_interval(data, false);
return;
+ }
+
+ adjust_xid_advancement_interval(data, true);
data->candidate_xid = full_xid;
data->phase = RCI_REQUEST_PUBLISHER_STATUS;
@@ -4156,6 +4181,41 @@ get_candidate_xid(RetainConflictInfoData *data)
maybe_advance_nonremovable_xid(data, false);
}
+/*
+ * Adjust the interval for advancing non-removable transaction IDs.
+ *
+ * If no new transaction ID has been assigned since the last advancement, the
+ * interval is doubled. This increase is limited by the
+ * wal_receiver_status_interval if it is not zero, or otherwise restricted to a
+ * maximum of 3 minutes. If a new transaction ID is detected, the interval is
+ * reset to a minimum of 100ms.
+ */
+static void
+adjust_xid_advancement_interval(RetainConflictInfoData *data, bool new_xid_found)
+{
+ if (!new_xid_found && data->xid_advancement_interval)
+ {
+ int max_interval = wal_receiver_status_interval
+ ? wal_receiver_status_interval * 1000
+ : MAX_XID_ADVANCEMENT_INTERVAL;
+
+ /*
+ * No new transaction ID assigned since the last check, so double the
+ * interval, but not beyond the maximum allowable value.
+ */
+ data->xid_advancement_interval = Min(data->xid_advancement_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_advancement_interval = MIN_XID_ADVANCEMENT_INTERVAL;
+ }
+}
+
/*
* Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
*/
--
2.30.0.windows.2
v16-0003-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v16-0003-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 623a2d587e5c2fc583459281ba8d8e2e45f2df99 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v16 3/6] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 16f26866f2..07d0ca477d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1128,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1156,6 +1168,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();
@@ -1176,14 +1190,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1216,6 +1259,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1243,6 +1315,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 4a206f9527..937afa5b6a 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..8c18264f58 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v16-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v16-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 69dc4ce13569839043699ca7577246cff89a7ff0 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:49:03 +0800
Subject: [PATCH v16 4/6] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 146 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++---
src/backend/utils/adt/pgstatfuncs.c | 10 +-
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 +-
11 files changed, 240 insertions(+), 31 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..41ae3e2a07 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1640,6 +1640,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-delete-origin-differs" xreflabel="delete_origin_differs">
<term><literal>delete_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..8f7d6f4d46 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2221,6 +2221,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_delete_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..3704d1b841 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
ss.stats_reset
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 68deea50f6..01cbbe3cd1 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -312,7 +315,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -337,6 +340,12 @@ 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, columns))
+ continue;
+
/*
* If one value is NULL and other is not, then they are certainly not
* equal
@@ -412,7 +421,7 @@ retry:
/* 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;
@@ -463,6 +472,135 @@ retry:
return found;
}
+/*
+ * Build a bitmap that includes the column numbers for the provided index.
+ */
+static Bitmapset *
+build_index_column_bitmap(Oid indexoid)
+{
+ IndexInfo *indexinfo;
+ Bitmapset *index_bitmap = NULL;
+ Relation idxrel;
+
+ Assert(CheckRelationOidLockedByMe(indexoid, RowExclusiveLock, true));
+
+ /* Open the index. */
+ idxrel = index_open(indexoid, NoLock);
+
+ indexinfo = BuildIndexInfo(idxrel);
+
+ for (int i = 0; i < indexinfo->ii_NumIndexAttrs; i++)
+ {
+ int keycol = indexinfo->ii_IndexAttrNumbers[i];
+
+ index_bitmap = bms_add_member(index_bitmap, keycol);
+ }
+
+ index_close(idxrel, NoLock);
+
+ return index_bitmap;
+}
+
+/*
+ * 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 = NULL;
+ Oid indexoid;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_origin = InvalidRepOriginId;
+ *delete_time = 0;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ oldestXmin = GetOldestNonRemovableTransactionId(rel);
+ indexoid = RelationGetReplicaIndex(rel);
+
+ if (OidIsValid(indexoid))
+ indexbitmap = build_index_column_bitmap(indexoid);
+
+ eq = palloc0(sizeof(*eq) * searchslot->tts_tupleDescriptor->natts);
+
+ /* Start a heap scan. */
+ 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);
+
+ 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 5d9ff626bd..cd39727932 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -28,6 +28,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_ORIGIN_DIFFERS] = "update_origin_differs",
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
[CT_DELETE_MISSING] = "delete_missing"
};
@@ -172,6 +173,7 @@ errcode_apply_conflict(ConflictType type)
return errcode(ERRCODE_UNIQUE_VIOLATION);
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
+ case CT_UPDATE_DELETED:
case CT_DELETE_ORIGIN_DIFFERS:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
@@ -259,6 +261,26 @@ errdetail_apply_conflict(EState *estate, ResultRelInfo *relinfo,
appendStringInfo(&err_detail, _("Could not find the row to be updated."));
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_DELETE_ORIGIN_DIFFERS:
if (localorigin == InvalidRepOriginId)
appendStringInfo(&err_detail, _("Deleting 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 2cbffa6db3..a2abb65abc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2738,6 +2738,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2754,15 +2757,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2773,7 +2772,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2792,19 +2791,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3126,7 +3133,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3136,17 +3143,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3158,7 +3173,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3169,7 +3184,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 60a397dc56..82da67e177 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1984,7 +1984,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2012,11 +2012,13 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_delete_origin_differs",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 8, "confl_update_deleted",
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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 9575524007..91470bac84 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_update_deleted,confl_delete_origin_differs,confl_delete_missing,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 494ec4f2e5..7d7fe5f4b3 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"
@@ -672,7 +673,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 c759677ff5..d2d3e2f064 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -35,6 +35,9 @@ typedef enum
/* The row to be updated is missing */
CT_UPDATE_MISSING,
+ /* The row to be updated is deleted */
+ CT_UPDATE_DELETED,
+
/* The row to be deleted was modified by a different origin */
CT_DELETE_ORIGIN_DIFFERS,
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 3014d047fe..8ed12428e1 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,11 +2147,12 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_update_origin_differs,
ss.confl_update_exists,
ss.confl_update_missing,
+ ss.confl_update_deleted,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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, stats_reset);
+ 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_update_deleted, confl_delete_origin_differs, confl_delete_missing, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v16-0005-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v16-0005-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From 5e9864f47ca20b16c742cc3e9384b35941fc3135 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:02:46 +0800
Subject: [PATCH v16 5/6] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 362 insertions(+), 116 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bf3cee08a9..6e2aa8a9a0 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8052,6 +8052,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 41ae3e2a07..d176e98f84 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1647,7 +1647,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 3704d1b841..a270d51c28 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 01cbbe3cd1..53a8a8eebc 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -535,14 +535,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
oldestXmin = GetOldestNonRemovableTransactionId(rel);
indexoid = RelationGetReplicaIndex(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 07d0ca477d..ecd11e9508 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index a2abb65abc..cc4e0c2fbf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2738,9 +2738,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2794,7 +2794,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3132,9 +3133,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3146,7 +3147,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3163,8 +3165,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4113,6 +4115,10 @@ static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* It is sufficient to manage non-removable transaction ID for a
* subscription by the main apply worker to detect update_deleted conflict
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index ec0cdf4ed7..9bcae370a6 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5009,6 +5017,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].subdetectupdatedeleted =
+ (strcmp(PQgetvalue(res, i, i_subdetectupdatedeleted), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5267,6 +5277,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subdetectupdatedeleted)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 2e55a0e3bb..55e32cdff5 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -672,6 +672,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subdetectupdatedeleted;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 2657abdc72..2f8b911574 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index beaff6578a..a811b4a7d1 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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
Dear hackers,
I did some benchmarks with the patch. More detail, a pub-sub replication system
was built and TPS was measured on the subscriber. Results were shown that the
performance can be degraded if the wal_receiver_status_interval is long.
This is expected behavior because the patch retains more dead tuples on the
subscriber side.
Also, we've considered a new mechanism which dynamically tunes the period of status
request, and confirmed that it reduced the regression. This is described the latter
part.
Below part contained the detailed report.
## Motivation - why the benchmark is needed
V15 patch set introduces a new replication slot on the subscriber side to retain
needed tuples for the update_deleted detection.
However, this may affect the performance of query executions on the subscriber because
1) tuples to be scaned will be increased and 2) HOT update cannot be worked.
The second issue comes from the fact that HOT update can work only when both tuples
can be located on the same page.
Based on above reasons I ran benchmark tests for the subscriber. The variable of the
measurement is the wal_receiver_status_interval, which controls the duration of status request.
## Used source code
HEAD was 962da900, and applied v15 patch set atop it.
## Environment
RHEL 7 machine which has 755GB memory, 4 physical CPUs and 120 logical processors.
## Workload
1. Constructed a pub-sub replication system.
Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
track_commit_timestamp = on (only for subscriber)
2. Ran pgbench with initialize mode. The scale factor was set to 100.
3. Ran pgbench with 60 clients for the subscriber. The duration was 120s,
and the measurement was repeated 5 times.
Attached script can automate above steps. You can specify the source type in the
measure.sh and run it.
## Comparison
The performance testing was done for HEAD and patched source code.
In case of patched, "detect_update_deleted" parameter was set to on. Also, the
parameter "wal_receiver_status_interval" was varied to 1s, 10s, and 100s to
check the effect.
Appendix table shows results [1]. The regression becomes larger based on the wal_receiver_status_interval.
TPS regressions are almost 5%(interval=1s) -> 25%(intervals=10s) -> 55% (intervals=55%).
Attached png file visualize the result: each bar shows median.
## Analysis
I attached to the backend via perf and found that heapam_index_fetch_tuple()
consumed much CPU time ony in case of patched [2]. Also, I checked pg_stat_all_tables
view and found that HOT update rarely happened only in the patched case [3].
This means that whether backend could do HOT update is the dominant.
When the detect_update_deleted = on, the additional slot is defined on the subscriber
ide and it is updated based on the activity; The frequency is determined by the
wal_receiver_status_intervals. In intervals=100s case, it is relatively larger
for the workload so that some dead tuples remained, this makes query processing slower.
This result means that users may have to tune consider the interval period based
on their workload. However, it is difficult to predict the appropriate value.
## Experiment - dynamic period tuning
Based on above, I and Hou discussed off-list and implemented new mechanism which
tunes the duration between status request dynamically. The basic idea is similar
with what slotsync worker does. The interval of requests is initially 100ms,
and becomes twice when if there are no XID assignment since the last advancement.
The maxium value is either of wal_receiver_status_interval or 3min.
Benchmark results with this are shown in [4]. Here wal_receiver_status_interval
is not changed, so we can compare with the HEAD and interval=10s case in [1] - 59536 vs 59597.
The regression is less than 1%.
The idea has already been included in v16-0002, please refer it.
## Experiment - shorter interval
Just in case - I did an extreme case that wal_receiver_status_interval is quite short - 10ms.
To make interval shorter I implemented an attached patch for both cases. Results are shown [5].
The regression is not observed or even better (I think this is caused by the randomness).
This experiment also shows the result that the regression is happened due to the dead tuple.
## Appendix [1] - result table
Each cells show transaction per seconds of the run.
patched
# run interval=1s intervals=10s intervals=100s
1 55876 45288 26956
2 56086 45336 26799
3 56121 45129 26753
4 56310 45169 26542
5 55389 45071 26735
median 56086 45169 26753
HEAD
# run interval=1s intervals=10s intervals=100s
1 59096 59343 59341
2 59671 59413 59281
3 59131 59597 58966
4 59239 59693 59518
5 59165 59631 59487
median 59165 59597 59341
## Appendix [2] - perf analysis
patched:
```
- 58.29% heapam_index_fetch_tuple
+ 38.28% heap_hot_search_buffer
+ 13.88% ReleaseAndReadBuffer
5.34% heap_page_prune_opt
+ 13.88% ReleaseAndReadBuffer
```
head:
```
- 2.13% heapam_index_fetch_tuple
1.06% heap_hot_search_buffer
0.62% heap_page_prune_opt
```
## Appendix [3] - pg_stat
patched
```
postgres=# SELECT relname, n_tup_upd, n_tup_hot_upd, n_tup_newpage_upd, n_tup_upd - n_tup_hot_upd AS n_tup_non_hot FROM pg_stat_all_tables where relname like 'pgbench%';
relname | n_tup_upd | n_tup_hot_upd | n_tup_newpage_upd | n_tup_non_hot
------------------+-----------+---------------+-------------------+---------------
pgbench_history | 0 | 0 | 0 | 0
pgbench_tellers | 453161 | 37996 | 415165 | 415165
pgbench_accounts | 453161 | 0 | 453161 | 453161
pgbench_branches | 453161 | 272853 | 180308 | 180308
(4 rows)
```
head
```
postgres=# SELECT relname, n_tup_upd, n_tup_hot_upd, n_tup_newpage_upd, n_tup_upd - n_tup_hot_upd AS n_tup_non_hot FROM
pg_stat_all_tables where relname like 'pgbench%';
relname | n_tup_upd | n_tup_hot_upd | n_tup_newpage_upd | n_tup_non_hot
------------------+-----------+---------------+-------------------+---------------
pgbench_history | 0 | 0 | 0 | 0
pgbench_tellers | 2078197 | 2077583 | 614 | 614
pgbench_accounts | 2078197 | 1911535 | 166662 | 166662
pgbench_branches | 2078197 | 2078197 | 0 | 0
(4 rows)
```
## Appendix [4] - dynamic status request
# run dynamic (v15 + PoC)
1 59627
2 59536
3 59359
4 59443
5 59541
median 59536
## Apendix [5] - shorter wal_receiver_status_interval
pached
# run interval=10ms
1 58081
2 57876
3 58083
4 57915
5 57933
median 57933
head
# run interval=10ms
1 57595
2 57322
3 57271
4 57421
5 57590
median 57421
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Attachments:
change_to_ms.diffsapplication/octet-stream; name=change_to_ms.diffsDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index e0c8325a39..f211ad5a1d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5007,7 +5007,7 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
or when <varname>synchronous_commit</varname> is set to
<literal>remote_apply</literal>.
Thus, the apply position may lag slightly behind the true position.
- If this value is specified without units, it is taken as seconds.
+ If this value is specified without units, it is taken as milliseconds.
The default value is 10 seconds. This parameter can only be set in
the <filename>postgresql.conf</filename> file or on the server
command line.
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 4657612a2f..494d69dc24 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3981,7 +3981,7 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
writepos == last_writepos &&
flushpos == last_flushpos &&
!TimestampDifferenceExceeds(send_time, now,
- wal_receiver_status_interval * 1000))
+ wal_receiver_status_interval))
return;
send_time = now;
@@ -4134,7 +4134,7 @@ get_candidate_xid(RetainConflictInfoData *data)
* consider the other interval or a separate GUC if the need arises.
*/
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
- wal_receiver_status_interval * 1000))
+ wal_receiver_status_interval))
return;
data->candidate_xid_time = now;
@@ -4310,7 +4310,7 @@ can_advance_nonremovable_xid(RetainConflictInfoData *data, TimestampTz now)
return MySubscription->detectupdatedeleted &&
data->phase == RCI_GET_CANDIDATE_XID &&
TimestampDifferenceExceeds(data->candidate_xid_time, now,
- wal_receiver_status_interval * 1000);
+ wal_receiver_status_interval);
}
/*
diff --git a/src/backend/replication/walreceiver.c b/src/backend/replication/walreceiver.c
index 5f641d2790..678118f076 100644
--- a/src/backend/replication/walreceiver.c
+++ b/src/backend/replication/walreceiver.c
@@ -1335,13 +1335,13 @@ WalRcvComputeNextWakeup(WalRcvWakeupReason reason, TimestampTz now)
if (!hot_standby_feedback || wal_receiver_status_interval <= 0)
wakeup[reason] = TIMESTAMP_INFINITY;
else
- wakeup[reason] = TimestampTzPlusSeconds(now, wal_receiver_status_interval);
+ wakeup[reason] = TimestampTzPlusMilliseconds(now, wal_receiver_status_interval);
break;
case WALRCV_WAKEUP_REPLY:
if (wal_receiver_status_interval <= 0)
wakeup[reason] = TIMESTAMP_INFINITY;
else
- wakeup[reason] = TimestampTzPlusSeconds(now, wal_receiver_status_interval);
+ wakeup[reason] = TimestampTzPlusMilliseconds(now, wal_receiver_status_interval);
break;
/* there's intentionally no default: here */
}
diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c
index 8cf1afbad2..4774e06415 100644
--- a/src/backend/utils/misc/guc_tables.c
+++ b/src/backend/utils/misc/guc_tables.c
@@ -2252,10 +2252,10 @@ struct config_int ConfigureNamesInt[] =
{"wal_receiver_status_interval", PGC_SIGHUP, REPLICATION_STANDBY,
gettext_noop("Sets the maximum interval between WAL receiver status reports to the sending server."),
NULL,
- GUC_UNIT_S
+ GUC_UNIT_MS
},
&wal_receiver_status_interval,
- 10, 0, INT_MAX / 1000,
+ 10 * 1000, 0, INT_MAX,
NULL, NULL, NULL
},
diff --git a/src/test/recovery/t/001_stream_rep.pl b/src/test/recovery/t/001_stream_rep.pl
index f3ea45ac4a..ec57b5fde1 100644
--- a/src/test/recovery/t/001_stream_rep.pl
+++ b/src/test/recovery/t/001_stream_rep.pl
@@ -344,7 +344,7 @@ is( $node_primary->psql(
$node_standby_1->append_conf('postgresql.conf',
"primary_slot_name = $slotname_1");
$node_standby_1->append_conf('postgresql.conf',
- "wal_receiver_status_interval = 1");
+ "wal_receiver_status_interval = 1s");
$node_standby_1->append_conf('postgresql.conf', "max_replication_slots = 4");
$node_standby_1->restart;
is( $node_standby_1->psql(
@@ -355,7 +355,7 @@ is( $node_standby_1->psql(
$node_standby_2->append_conf('postgresql.conf',
"primary_slot_name = $slotname_2");
$node_standby_2->append_conf('postgresql.conf',
- "wal_receiver_status_interval = 1");
+ "wal_receiver_status_interval = 1s");
# should be able change primary_slot_name without restart
# will wait effect in get_slot_xmins above
$node_standby_2->reload;
diff --git a/src/test/recovery/t/010_logical_decoding_timelines.pl b/src/test/recovery/t/010_logical_decoding_timelines.pl
index afcd5241aa..ab9a403aeb 100644
--- a/src/test/recovery/t/010_logical_decoding_timelines.pl
+++ b/src/test/recovery/t/010_logical_decoding_timelines.pl
@@ -42,7 +42,7 @@ max_replication_slots = 3
max_wal_senders = 2
log_min_messages = 'debug2'
hot_standby_feedback = on
-wal_receiver_status_interval = 1
+wal_receiver_status_interval = 1s
]);
$node_primary->dump_info;
$node_primary->start;
On Wed, Dec 11, 2024 at 2:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V16 patch set which addressed above comments.
Review Comments
================
1.
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *data)
+{
...
+ /*
+ * 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ */
+ if (!AllTablesyncsReady())
+ return;
How is it ensured that new tables are not added to subscription via
refresh immediately after this check that are not yet in ready state?
I mean if it happens immediately after this check then the problem
described by comment can happen and we may end up advancing the
non-removable-xid incorrectly. If this is safe, then please update
comments to reflect the same.
2.
+ /*
+ * Reset all data fields except those used to determine the timing for the
+ * next round of transaction ID advancement.
+ */
+ memset(data, 0, offsetof(RetainConflictInfoData, candidate_xid_time));
Wouldn't it be better to initialize each field separately? With the
current code, adding new fields at the end of the structure
RetainConflictInfoData will be difficult.
3.
+ * TODO: The remote flush location (last_flushpos) is currently not updated
+ * during change application, making it impossible to satisfy the condition of
+ * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
+ * Consider updating the remote flush position in the final phase to enable
+ * advancement during change application.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
I think we don't need this TODO here as there is XXX comment in
wait_for_local_flush() which has the same information.
4.
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'S':
+ ProcessStandbyPSRequestMessage();
+ break;
Why do we use the capital message name 'S' when other messages in
ProcessStandbyMessage() are all small cases? I see that walsender
already handles 'S' message in HandleUploadManifestPacket() though it
won't conflict with this case. But still, shouldn't we use a different
message here?
5. The apply worker needs to at least twice get the publisher status
message to advance oldest_nonremovable_xid once. It then uses the
remote_lsn of the last such message to ensure that it has been applied
locally. Such a remote_lsn could be a much later value than required
leading to delay in advancing oldest_nonremovable_xid. How about if
while first time processing the publisher_status message on walsender,
we get the latest_transaction_in_commit by having a function
GetLatestTransactionIdInCommit() instead of
GetOldestTransactionIdInCommit() and then simply wait till that proc
has written commit WAL (aka wait till it clears
DELAY_CHKPT_IN_COMMIT)? Then get the latest LSN wrote and send that to
apply worker waiting for the publisher_status message. If this is
feasible then we should be able to advance oldest_nonremovable_xid
with just one publisher_status message. Won't that be an improvement
over current? If so, we can even further try to improve it by just
using commit_LSN of the transaction returned by
GetLatestTransactionIdInCommit(). One idea is that we can try to use
MyProc->waitLSN which we are using in synchronous replication for our
purpose. See SyncRepWaitForLSN.
6. Attached, a few minor comment updates.
--
With Regards,
Amit Kapila.
Attachments:
v16-0001-amit.1.patch.txttext/plain; charset=US-ASCII; name=v16-0001-amit.1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6b79aa441b..afe007bd50 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4312,12 +4312,6 @@ wait_for_local_flush(RetainConflictInfoData *data)
/*
* Determine if we can attempt to advance transaction ID.
- *
- * TODO: The remote flush location (last_flushpos) is currently not updated
- * during change application, making it impossible to satisfy the condition of
- * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
- * Consider updating the remote flush position in the final phase to enable
- * advancement during change application.
*/
static inline bool
can_advance_nonremovable_xid(RetainConflictInfoData *data)
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 00b6411c7e..94acca15e3 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -2678,6 +2678,10 @@ ProcessStandbyPSRequestMessage(void)
WalSnd *walsnd = MyWalSnd;
TimestampTz replyTime;
+ /*
+ * This shouldn't happen because we don't support getting publisher_status
+ * message from standby.
+ */
if (RecoveryInProgress())
elog(ERROR, "the primary status is unavailable during recovery");
On Wed, Dec 11, 2024 at 2:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V16 patch set which addressed above comments.
There is a new 0002 patch where I tried to dynamically adjust the interval for
advancing the transaction ID. Instead of always waiting for
wal_receiver_status_interval, we can start with a short interval and increase
it if there is no activity (no xid assigned on subscriber), but not beyond
wal_receiver_status_interval.The intention is to more effectively advance xid to avoid retaining too much
dead tuples. My colleague will soon share detailed performance data and
analysis related to this enhancement.
I am starting to review the patches, and trying to understand the
concept that how you are preventing vacuum to remove the dead tuple
which might required by the concurrent remote update, so I was looking
at the commit message which explains the idea quite clearly but I have
one question
The process of advancing the non-removable transaction ID in the apply worker
involves:
== copied from commit message of 0001 start==
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.
== copied from commit message of 0001 start==
So IIUC in step 2) we send the message and get the list of all the
transactions which are in the commit phase? What do you exactly mean
by a transaction which is in the commit phase? Can I assume
transactions which are currently running on the publisher? And in
step 3) we wait for all the transactions to get committed which we saw
running (or in the commit phase) and we anyway don't worry about the
newly started transactions as they would not be problematic for us.
And in step 4) we would wait for all the flush location to reach "last
received WAL position", here my question is what exactly will be the
"last received WAL position" I assume it would be the position
somewhere after the position of the commit WAL of all the transaction
we were interested on the publisher?
At high level the overall idea looks promising to me but wanted to put
more thought on lower level details about what transactions exactly we
are waiting for and what WAL LSN we are waiting to get flushed.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Monday, December 16, 2024 7:21 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
Hi,
On Wed, Dec 11, 2024 at 2:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the V16 patch set which addressed above comments.
There is a new 0002 patch where I tried to dynamically adjust the interval for
advancing the transaction ID. Instead of always waiting for
wal_receiver_status_interval, we can start with a short interval and increase
it if there is no activity (no xid assigned on subscriber), but not beyond
wal_receiver_status_interval.The intention is to more effectively advance xid to avoid retaining too much
dead tuples. My colleague will soon share detailed performance data and
analysis related to this enhancement.I am starting to review the patches, and trying to understand the
concept that how you are preventing vacuum to remove the dead tuple
which might required by the concurrent remote update, so I was looking
at the commit message which explains the idea quite clearly but I have
one question
Thanks for the review!
The process of advancing the non-removable transaction ID in the apply worker
involves:== copied from commit message of 0001 start==
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.
== copied from commit message of 0001 start==So IIUC in step 2) we send the message and get the list of all the
transactions which are in the commit phase? What do you exactly mean by a
transaction which is in the commit phase?
I was referring to transactions calling RecordTransactionCommit() and have
entered the commit critical section. In the patch, we checked if the proc has
marked the new flag DELAY_CHKPT_IN_COMMIT in 'MyProc->delayChkptFlags'.
Can I assume transactions which are currently running on the publisher?
I think it's a subset of the running transactions. We only get the transactions
in commit phase with the intention to avoid delays caused by waiting for
long-running transactions to complete, which can result in the long retention
of dead tuples.
We decided to wait for running(committing) transactions due to the WAL/LSN
inversion issue[1]/messages/by-id/OS0PR01MB571628594B26B4CC2346F09294592@OS0PR01MB5716.jpnprd01.prod.outlook.com>. The original idea is to directly return the latest WAL
write position without checking running transactions. But since there is a gap
between when we acquire the commit_timestamp and the commit LSN, it's possible
the transactions might have been assigned an earlier commit timestamp but have
not yet written the commit WAL record.
And in step 3) we wait for all the transactions to get committed which we saw
running (or in the commit phase) and we anyway don't worry about the newly
started transactions as they would not be problematic for us. And in step 4)
we would wait for all the flush location to reach "last received WAL
position", here my question is what exactly will be the "last received WAL
position" I assume it would be the position somewhere after the position of
the commit WAL of all the transaction we were interested on the publisher?
Yes, your understanding is correct. It's a position after the position of all
the interesting transactions. In the patch, we get the latest WAL write
position(GetXLogWriteRecPtr()) in walsender after all interesting transactions
have finished and reply it to apply worker.
At high level the overall idea looks promising to me but wanted to put
more thought on lower level details about what transactions exactly we
are waiting for and what WAL LSN we are waiting to get flushed.
Yeah, that makes sense, thanks.
[1]: /messages/by-id/OS0PR01MB571628594B26B4CC2346F09294592@OS0PR01MB5716.jpnprd01.prod.outlook.com>
Best Regards,
Hou zj
On Tue, Dec 17, 2024 at 8:54 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 16, 2024 7:21 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
So IIUC in step 2) we send the message and get the list of all the
transactions which are in the commit phase? What do you exactly mean by a
transaction which is in the commit phase?I was referring to transactions calling RecordTransactionCommit() and have
entered the commit critical section. In the patch, we checked if the proc has
marked the new flag DELAY_CHKPT_IN_COMMIT in 'MyProc->delayChkptFlags'.Can I assume transactions which are currently running on the publisher?
I think it's a subset of the running transactions. We only get the transactions
in commit phase with the intention to avoid delays caused by waiting for
long-running transactions to complete, which can result in the long retention
of dead tuples.
Ok
We decided to wait for running(committing) transactions due to the WAL/LSN
inversion issue[1]. The original idea is to directly return the latest WAL
write position without checking running transactions. But since there is a gap
between when we acquire the commit_timestamp and the commit LSN, it's possible
the transactions might have been assigned an earlier commit timestamp but have
not yet written the commit WAL record.
Yes, that makes sense.
And in step 3) we wait for all the transactions to get committed which we saw
running (or in the commit phase) and we anyway don't worry about the newly
started transactions as they would not be problematic for us. And in step 4)
we would wait for all the flush location to reach "last received WAL
position", here my question is what exactly will be the "last received WAL
position" I assume it would be the position somewhere after the position of
the commit WAL of all the transaction we were interested on the publisher?Yes, your understanding is correct. It's a position after the position of all
the interesting transactions. In the patch, we get the latest WAL write
position(GetXLogWriteRecPtr()) in walsender after all interesting transactions
have finished and reply it to apply worker.
Got it, thanks.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Sunday, December 15, 2024 9:39 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 11, 2024 at 2:32 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the V16 patch set which addressed above comments.
Review Comments ================ 1. +/* + * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase. + */ +static void +wait_for_local_flush(RetainConflictInfoData *data) { ... + /* + * 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. In this case, confirming the apply and flush + * progress across all table sync workers is complex and not worth the + * effort. + */ + if (!AllTablesyncsReady()) + return;How is it ensured that new tables are not added to subscription via refresh
immediately after this check that are not yet in ready state?
I mean if it happens immediately after this check then the problem described
by comment can happen and we may end up advancing the non-removable-xid
incorrectly. If this is safe, then please update comments to reflect the same.
I think it's safe because WAL positions of changes from these new tables, which
will be applied, should be greater than remote_lsn and are included in
transactions with later commit timestamps. So, we do not need to wait for these
changes to be applied in this round of advancement.
And I added above in the comments.
2. + /* + * Reset all data fields except those used to determine the timing for + the + * next round of transaction ID advancement. + */ + memset(data, 0, offsetof(RetainConflictInfoData, candidate_xid_time));Wouldn't it be better to initialize each field separately? With the current code,
adding new fields at the end of the structure RetainConflictInfoData will be
difficult.
Changed as suggested.
3. + * TODO: The remote flush location (last_flushpos) is currently not +updated + * during change application, making it impossible to satisfy the +condition of + * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID. + * Consider updating the remote flush position in the final phase to +enable + * advancement during change application. + */ +static inline bool +can_advance_nonremovable_xid(RetainConflictInfoData *data)I think we don't need this TODO here as there is XXX comment in
wait_for_local_flush() which has the same information.
Removed.
4.
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;+ case 'S': + ProcessStandbyPSRequestMessage(); + break;Why do we use the capital message name 'S' when other messages in
ProcessStandbyMessage() are all small cases? I see that walsender already
handles 'S' message in HandleUploadManifestPacket() though it won't conflict
with this case. But still, shouldn't we use a different message here?
OK, I chose a new name 'p' in the latest version.
5. The apply worker needs to at least twice get the publisher status message to
advance oldest_nonremovable_xid once. It then uses the remote_lsn of the last
such message to ensure that it has been applied locally. Such a remote_lsn
could be a much later value than required leading to delay in advancing
oldest_nonremovable_xid. How about if while first time processing the
publisher_status message on walsender, we get the
latest_transaction_in_commit by having a function
GetLatestTransactionIdInCommit() instead of
GetOldestTransactionIdInCommit() and then simply wait till that proc has
written commit WAL (aka wait till it clears DELAY_CHKPT_IN_COMMIT)?
Then get the latest LSN wrote and send that to apply worker waiting for the
publisher_status message. If this is feasible then we should be able to
advance oldest_nonremovable_xid with just one publisher_status message.
Won't that be an improvement over current? If so, we can even further try to
improve it by just using commit_LSN of the transaction returned by
GetLatestTransactionIdInCommit(). One idea is that we can try to use
MyProc->waitLSN which we are using in synchronous replication for our
purpose. See SyncRepWaitForLSN.
I will do more performance tests on this and address if it improves
the performance.
6. Attached, a few minor comment updates.
Thanks. I have merged them.
Attach the V17 patch set which addressed above comments.
Best Regards,
Hou zj
Attachments:
v17-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v17-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From d1d45803d7dda71377456c0962306a29708f3b1b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:51:23 +0800
Subject: [PATCH v17 6/6] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/035_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/035_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index b2395e7b57..a80c7f1656 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_confl_update_deleted.pl b/src/test/subscription/t/035_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/035_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
v17-0002-Dynamically-adjust-xid-advancement-interval.patchapplication/octet-stream; name=v17-0002-Dynamically-adjust-xid-advancement-interval.patchDownload
From 0de9a0d26630b6591f0c11ded47d021929d11590 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 16:37:00 +0800
Subject: [PATCH v17 2/6] Dynamically adjust xid advancement interval
If no new transaction ID has been assigned since the last advancement, the
interval is doubled. This increase is limited by the
wal_receiver_status_interval if it is not zero, or otherwise restricted to a
maximum of 3 minutes. If a new transaction ID is detected, the interval is
reset to a minimum of 100ms.
---
src/backend/replication/logical/worker.c | 62 +++++++++++++++++++++++-
1 file changed, 61 insertions(+), 1 deletion(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 654078a726..8cd81de908 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -313,8 +313,19 @@ typedef struct RetainConflictInfoData
TimestampTz reply_time; /* when the publisher responds with status */
TimestampTz last_recv_time; /* when the last message was received */
TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+ int xid_advancement_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.
+ */
+#define MIN_XID_ADVANCEMENT_INTERVAL 100
+#define MAX_XID_ADVANCEMENT_INTERVAL 180000L
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -428,6 +439,8 @@ 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 adjust_xid_advancement_interval(RetainConflictInfoData *data,
+ bool new_xid_found);
static void request_publisher_status(RetainConflictInfoData *data);
static void wait_for_publisher_status(RetainConflictInfoData *data,
bool status_received);
@@ -3835,6 +3848,13 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to attempt advancing the
+ * non-removable transaction ID.
+ */
+ if (data.phase == RCI_GET_CANDIDATE_XID && data.xid_advancement_interval)
+ wait_time = Min(wait_time, data.xid_advancement_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -4129,7 +4149,7 @@ get_candidate_xid(RetainConflictInfoData *data)
* can consider the other interval or a separate GUC if the need arises.
*/
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
- wal_receiver_status_interval * 1000))
+ data->xid_advancement_interval))
return;
data->candidate_xid_time = now;
@@ -4147,7 +4167,12 @@ get_candidate_xid(RetainConflictInfoData *data)
/* Return if the oldest_nonremovable_xid cannot be advanced */
if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
full_xid))
+ {
+ adjust_xid_advancement_interval(data, false);
return;
+ }
+
+ adjust_xid_advancement_interval(data, true);
data->candidate_xid = full_xid;
data->phase = RCI_REQUEST_PUBLISHER_STATUS;
@@ -4156,6 +4181,41 @@ get_candidate_xid(RetainConflictInfoData *data)
maybe_advance_nonremovable_xid(data, false);
}
+/*
+ * Adjust the interval for advancing non-removable transaction IDs.
+ *
+ * If no new transaction ID has been assigned since the last advancement, the
+ * interval is doubled. This increase is limited by the
+ * wal_receiver_status_interval if it is not zero, or otherwise restricted to a
+ * maximum of 3 minutes. If a new transaction ID is detected, the interval is
+ * reset to a minimum of 100ms.
+ */
+static void
+adjust_xid_advancement_interval(RetainConflictInfoData *data, bool new_xid_found)
+{
+ if (!new_xid_found && data->xid_advancement_interval)
+ {
+ int max_interval = wal_receiver_status_interval
+ ? wal_receiver_status_interval * 1000
+ : MAX_XID_ADVANCEMENT_INTERVAL;
+
+ /*
+ * No new transaction ID assigned since the last check, so double the
+ * interval, but not beyond the maximum allowable value.
+ */
+ data->xid_advancement_interval = Min(data->xid_advancement_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_advancement_interval = MIN_XID_ADVANCEMENT_INTERVAL;
+ }
+}
+
/*
* Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
*/
--
2.30.0.windows.2
v17-0003-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v17-0003-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 3f3ac15f9b40b819e3e23d533077681ec6fe9088 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v17 3/6] 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.
---
src/backend/replication/logical/launcher.c | 211 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 238 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 16f26866f2..07d0ca477d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1128,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1156,6 +1168,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();
@@ -1176,14 +1190,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1216,6 +1259,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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. */
@@ -1243,6 +1315,145 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ 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);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotSave();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 4a206f9527..937afa5b6a 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 6dc36e10aa..f53856cc90 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v17-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v17-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 0f93072f5c6a6e39d78d00f398b9cc44aa7802b0 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:49:03 +0800
Subject: [PATCH v17 4/6] 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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 124 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +--
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 +-
11 files changed, 221 insertions(+), 34 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..55b8fb5ec6 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 840d7f8161..a048a0e4e1 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..adfea5fd6b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1370,6 +1370,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 68deea50f6..473da67a2d 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -312,7 +315,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -337,6 +340,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
@@ -412,7 +423,7 @@ retry:
/* 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;
@@ -463,6 +474,111 @@ retry:
return found;
}
+/*
+ * 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;
+
+ /* Return 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. */
+ 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);
+
+ 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 5d9ff626bd..ed743aa1a2 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 8cd81de908..22b3e15a1c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2737,6 +2737,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2753,15 +2756,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2772,7 +2771,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2791,19 +2790,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3125,7 +3132,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3135,17 +3142,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3157,7 +3172,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3168,7 +3183,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 03dd8cd335..287e6d7b15 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -1998,7 +1998,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2020,17 +2020,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 0f22c21723..bdce7e2887 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 c8e6befca8..73a33eea1a 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"
@@ -673,7 +674,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 c759677ff5..93005c7267 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 3014d047fe..2f5d4c970c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,6 +2144,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,
@@ -2151,7 +2152,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v17-0005-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v17-0005-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From b43cb68f633e62d84425ecbffe392ac6a6f03eb3 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 19 Dec 2024 12:23:10 +0800
Subject: [PATCH v17 5/6] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 4 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 20 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 362 insertions(+), 116 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bf3cee08a9..6e2aa8a9a0 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8052,6 +8052,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 55b8fb5ec6..dd2e6f45c0 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1609,7 +1609,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..88c9a39cbd 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,26 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. To ensure efficient
+ cleanup of dead tuples that are no longer needed for detection,
+ <xref linkend="guc-wal-receiver-status-interval"/> must be set to a
+ non-zero value.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index adfea5fd6b..710564eb85 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 473da67a2d..ddd61fd150 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -507,14 +507,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
/* Get the cutoff xmin for HeapTupleSatisfiesVacuum */
oldestXmin = GetOldestNonRemovableTransactionId(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 07d0ca477d..ecd11e9508 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22b3e15a1c..735d9e5ec0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2737,9 +2737,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2793,7 +2793,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3131,9 +3132,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3145,7 +3146,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3162,8 +3164,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4112,6 +4114,10 @@ static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* It is sufficient to manage non-removable transaction ID for a
* subscription by the main apply worker to detect update_deleted conflict
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 19969e400f..9a52eb569c 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5009,6 +5017,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].subdetectupdatedeleted =
+ (strcmp(PQgetvalue(res, i, i_subdetectupdatedeleted), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5267,6 +5277,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subdetectupdatedeleted)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9c5ddd20cf..1813557a7a 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subdetectupdatedeleted;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 2657abdc72..2f8b911574 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index beaff6578a..a811b4a7d1 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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v17-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v17-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 4e0bf4adb594fdec37f7935ddf3c15b24621de98 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v17 1/6] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 417 ++++++++++++++++++++-
src/backend/replication/walsender.c | 54 +++
src/backend/storage/ipc/procarray.c | 59 +++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 649 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..eadfcaab09 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 3ebd7c4041..611926c7d2 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 e5fdca8bbf..16f26866f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -450,6 +450,7 @@ retry:
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 9e50c880f8..654078a726 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,44 @@ 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 last_recv_time; /* when the last message was received */
+ TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +379,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +425,15 @@ 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 inline bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3651,6 +3707,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);
@@ -3677,6 +3735,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3758,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);
@@ -3706,6 +3796,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)
{
/*
@@ -3803,6 +3898,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
@@ -3838,7 +3935,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;
@@ -3916,6 +4012,325 @@ 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 if concurrent remote transactions persist) ->
+ * 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and
+ * network resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * It is safe to add new tables with initial states to the subscription
+ * after this check because WAL positions of changes from these new
+ * tables, which will be applied, should be greater than remote_lsn and
+ * are included in transactions with later commit timestamps. So, there is
+ * no need to wait for these changes to be applied in this round of
+ * advancement.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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, now 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;
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
/*
* 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 371eef3ddd..6dc36e10aa 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,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 c769b1aa3e..ae6da9fd91 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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 cda21239cb..d034c523a7 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2470,6 +2470,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
Here is further performance test analysis with v16 patch-set.
In the test scenarios already shared on -hackers [1]/messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com, where pgbench was run
only on the publisher node in a pub-sub setup, no performance degradation
was observed on either node.
In contrast, when pgbench was run only on the subscriber side with
detect_update_deleted=on [2]/messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com, the TPS performance was reduced due to dead
tuple accumulation. This performance drop depended on the
wal_receiver_status_interval—larger intervals resulted in more dead tuple
accumulation on the subscriber node. However, after the improvement in
patch v16-0002, which dynamically tunes the status request, the default TPS
reduction was limited to only 1%.
We performed more benchmarks with the v16-patches where pgbench was run on
both the publisher and subscriber, focusing on TPS performance. To
summarize the key observations:
- No performance impact on the publisher as dead tuple accumulation does
not occur on the publisher.
- The performance is reduced on the subscriber side (TPS reduction (~50%)
[3]: Test with pgbench run on both publisher and subscriber.
detect_update_deleted=on.
- Performance reduction happens only on the subscriber side, as workload
on the publisher is pretty high and the apply workers must wait for the
amount of transactions with earlier timestamps to be applied and flushed
before advancing the non-removable XID to remove dead tuples.
- To validate this further, we modified the patch to check only each
transaction's commit_time and advance the non-removable XID if the
commit_time is greater than candidate_xid_time. The benchmark results[4]Test with modifying the patch as mentioned above. (attached top-up patch " 0001-wait-for-commit-time")
remained consistent, showing similar performance reduction. This confirms
that the performance impact on the subscriber side is a reasonable behavior
if we want to detect the update_deleted conflict reliably.
We have also tested similar scenarios in physical streaming replication, to
see the effect of enabling the hot_standby_feedback and
recovery_min_apply_delay. The benchmark results[5]Test for effects of enabling the hot_standby_feedback and recovery_min_apply_delay on a standby node. showed performance
reduction in these cases as well, though impact was less compared to the
update_deleted scenario because the physical walreceiver does not need to
wait for specified WAL to be applied before sending the hot standby
feedback message. However, as the recovery_min_apply_delay increased, a
similar TPS reduction (~50%) was observed, aligning with the behavior seen
in the update_deleted case.
Based on the above, I think the performance reduction observed with the
update_deleted patch is expected and necessary because the patch's main
goal is to retain dead tuples for reliable conflict detection. Reducing
this retention period would compromise the accuracy of update_deleted
detection.
The detailed benchmark results are as follow:
[1]: /messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com
/messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com
[2]: /messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com
/messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com
[3]: Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
track_commit_timestamp = on (only for subscriber)
-- Note: to avoid the update/delete_missing conflicts, tables to be
modified on the publisher side was renamed: pgbench_XXX -> pgbench_pub_XXX.
Test Run:
- Ran pgbench(read-write) on both the publisher and the subscriber with 30
clients for a duration of 120 seconds, collecting data over 5 runs.
- Note that pgbench was running for different tables on pub and sub.
(The scripts used for test "case1-2_measure.sh" and case1-2_setup.sh" are
attached).
Results:
Run# pub TPS sub TPS
1 32209 13704
2 32378 13684
3 32720 13680
4 31483 13681
5 31773 13813
median 32209 13684
regression 7% -53%
Perf analysis: shows time spent on heap table scans- ```
- 68.22% 0.09% postgres postgres [.] ExecModifyTable
...
- 62.86% index_getnext_slot
- 61.49% index_fetch_heap
- 61.40% heapam_index_fetch_tuple
+ 41.43% heap_hot_search_buffer
+ 13.11% ReleaseAndReadBuffer
6.15% heap_page_prune_opt
+ 1.33% index_getnext_tid
+ 4.35% ExecUpdate
```
~~~~
[4]: Test with modifying the patch as mentioned above. (attached top-up patch " 0001-wait-for-commit-time")
patch " 0001-wait-for-commit-time")
Test setup:
- Setup is the same as [1]/messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com, only with the code modified as "
0001-wait-for-commit-time" patch.
Test Run:
- Ran pgbench(read-write) on both the publisher and the subscriber with 30
clients for a duration of 120 seconds, collecting data over 5 runs.
- Note that pgbench was running for different tables on pub and sub.
(The scripts used for test "case1-2_measure.sh" and case1-2_setup.sh" are
attached).
Results:
Run# pub TPS sub TPS
1 29678.49354 13440.01222
2 29686.43844 13416.32116
3 29915.70886 13412.92197
4 29986.15481 13410.23707
5 30098.37795 13289.21383
Median 29915.70886 13412.92197
Regression 3% -53%
Perf analysis: shows time spent on heap table scans -
``` --56.35%--index_fetch_heap
--56.04%--heapam_index_fetch_tuple
--38.72%--heap_hot_search_buffer
--11.23%--ReleaseAndReadBuffer
```
Data from pg_stat:
```
postgres=# SELECT relname, n_tup_upd, n_tup_hot_upd, n_tup_newpage_upd,
n_tup_upd - n_tup_hot_upd AS n_tup_non_hot FROM pg_stat_all_tables where
relname like 'pgbench_%';
relname | n_tup_upd | n_tup_hot_upd | n_tup_newpage_upd |
n_tup_non_hot
------------------+-----------+---------------+-------------------+---------------
pgbench_history | 0 | 0 | 0 |
0
pgbench_tellers | 1286798 | 934880 | 351918 |
351918
pgbench_accounts | 1286798 | 28701 | 1258097 |
1258097
pgbench_branches | 1286797 | 1240577 | 46220 |
46220
(4 rows)
```
~~~~
[5]: Test for effects of enabling the hot_standby_feedback and recovery_min_apply_delay on a standby node.
recovery_min_apply_delay on a standby node.
Test setup:
- All tests done on pgHead (1f0de66ea2)
- Created one primary node with configuration -
autovacuum = false
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
- Initialized the primary node with pgbench initial data using scale=100
- Created a standby node for the primary node.
Test runs:
- Ran pgbench(read-write on primary and read-only only standby) with 30
clients for a duration of 120 seconds, collecting data over 5 runs for each
case.
(The scripts used for test "case3_measure.sh" and case3_setup.sh" are
attached).
Results:
There was no regression observed on standby's TPS during these runs.
a) Data for wal_receiver_status_interval=1s/10s(default)/100s with
hot_standby_feedback=ON
- The TPS on Primary when hot_standby_feedback=OFF
Run# Primary TPS
1 31482.74898
2 31396.02757
3 31418.47558
4 31485.24074
5 31489.70499
Median 31482.74898
- The TPS on Primary when hot_standby_feedback=ON with varying
wal_receiver_status_interval
(Each cell shows the TPS of the Primary node.)
Run# interval=1s intervals=10s
intervals=100s
1 31056.25701 27754.2588
17163.77088
2 30889.89889 27627.41654
17115.88696
3 30862.499 27579.77763
17188.18505
4 30980.84342 27753.04054
17153.41827
5 31105.98614 27849.44234
17225.25687
Median 30980.84342 27753.04054 17163.77088
Regression -2% -12% -45%
b) Data for recovery_min_apply_delay =30s/60s/90s/1h with
hot_standby_feedback=ON
(Each cell shows the TPS of the Primary node.)
Run# apply_del=30s apply_del=60s
apply_del=90s apply_del=1h
1 17630.58618 15331.08557
13999.01297 13927.10651
2 17650.56248 15364.88802
14039.52754 14011.92415
3 17679.18014 15314.12559
14060.43152 13996.42864
4 17693.89016 15296.77917
14052.06037 14037.38131
5 17691.42358 15323.18087
14038.56499 13996.08657
Median 17679.18014 15323.18087
14039.52754 13996.42864
Regression -44% -51%
-55% -56%
~~~~
--
Thanks,
Nisha
Attachments:
0001-wait-for-commit-time.patchapplication/octet-stream; name=0001-wait-for-commit-time.patchDownload
From 780da196890758f6bb2c160b0cab7a30ac9f799c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 17 Dec 2024 10:11:08 +0800
Subject: [PATCH] wait for commit time
---
src/backend/replication/logical/worker.c | 39 ++----------------------
1 file changed, 3 insertions(+), 36 deletions(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6406be4f17..893db32935 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -445,7 +445,6 @@ 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 inline bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
@@ -3771,8 +3770,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* to avoid accumulating dead rows when the worker is
* busy.
*/
- if (can_advance_nonremovable_xid(&data))
- maybe_advance_nonremovable_xid(&data, false);
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -4196,7 +4194,7 @@ get_candidate_xid(RetainConflictInfoData *data)
adjust_xid_advancement_interval(data, true);
data->candidate_xid = full_xid;
- data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+ data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
/* process the next phase */
maybe_advance_nonremovable_xid(data, false);
@@ -4327,22 +4325,6 @@ 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
@@ -4362,7 +4344,7 @@ wait_for_local_flush(RetainConflictInfoData *data)
* oldest_nonremovable_xid may be delayed. We can always update
* last_flushpos here if we notice such a delay.
*/
- if (last_flushpos < data->remote_lsn)
+ if (replorigin_session_origin_timestamp < data->candidate_xid_time)
return;
/*
@@ -4391,21 +4373,6 @@ wait_for_local_flush(RetainConflictInfoData *data)
maybe_advance_nonremovable_xid(data, false);
}
-/*
- * Determine if we can attempt to advance transaction ID.
- *
- * TODO: The remote flush location (last_flushpos) is currently not updated
- * during change application, making it impossible to satisfy the condition of
- * the final phase (RCI_WAIT_FOR_LOCAL_FLUSH) for advancing the transaction ID.
- * Consider updating the remote flush position in the final phase to enable
- * advancement during change application.
- */
-static inline bool
-can_advance_nonremovable_xid(RetainConflictInfoData *data)
-{
- return data->phase == RCI_GET_CANDIDATE_XID;
-}
-
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
--
2.30.0.windows.2
Dear Hou,
Thanks for updating the patch. Few comments:
01. worker.c
```
+/*
+ * The minimum (100ms) and maximum (3 minutes) intervals for advancing
+ * non-removable transaction IDs.
+ */
+#define MIN_XID_ADVANCEMENT_INTERVAL 100
+#define MAX_XID_ADVANCEMENT_INTERVAL 180000L
```
Since the max_interval is an integer variable, it can be s/180000L/180000/.
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points - create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into ReplicationSlotValidateName()?
03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```
Assuming the case that the launcher crashed just after ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.
I think we should re-create the slot when the xmin is invalid. Thought?
04. documentation
Should we update "Configuration Settings" section in logical-replication.sgml
because an additional slot is required?
05. check_remote_recovery()
Can we add a test case related with this?
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?
I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.
03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.
05. check_remote_recovery()
Can we add a test case related with this?
I think the code path is already tested, and I am a bit unsure if we want to setup
a standby to test the ERROR case, so didn't add this.
---
Attach the new version patch set which addressed all other comments.
Based on some off-list discussions with Sawada-san and Amit, it would be better
if the apply worker can avoid reporting an ERROR if the publisher's clock's
lags behind that of the subscriber, so I implemented a new 0007 patch to allow
the apply worker to wait for the clock skew to pass and then send a new request
to the publisher for the latest status. The implementation is as follows:
Since we have the time (reply_time) on the walsender when it confirms that all
the committing transactions have finished, it means any subsequent transactions
on the publisher should be assigned a commit timestamp later then reply_time.
And the (candidate_xid_time) when it determines the oldest active xid. Any old
transactions on the publisher that have finished should have a commit timestamp
earlier than the candidate_xid_time.
The apply worker can compare the candidate_xid_time with reply_time. If
candidate_xid_time is less than the reply_time, then it's OK to advance the xid
immdidately. If candidate_xid_time is greater than reply_time, it means the
clock of publisher is behind that of the subscriber, so the apply worker can
wait for the skew to pass before advancing the xid.
Since this is considered as an improvement, we can focus on this after
pushing the main patches.
Best Regards,
Hou zj
Attachments:
v18-0007-Wait-for-publisher-s-clock-to-catch-up-before-pr.patchapplication/octet-stream; name=v18-0007-Wait-for-publisher-s-clock-to-catch-up-before-pr.patchDownload
From 17b25efa8af1276922a7e4c9a8423c783019db80 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 24 Dec 2024 14:48:26 +0800
Subject: [PATCH v18 7/7] Wait for publisher's clock to catch up before
proceeding
When the publisher's clock is behind the subscriber's, instead of repeatedly
reporting an ERROR, allow the apply worker to wait for the clock skew to
resolve and then send a new request to the publisher for the latest status.
This approach allows for the detection of update_deleted conflicts even if
clock skew exists.
---
src/backend/replication/logical/worker.c | 71 ++++++++++++++++--------
src/backend/replication/walsender.c | 11 +++-
2 files changed, 59 insertions(+), 23 deletions(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 895aa4b9bc..f0b7653c35 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -313,6 +313,7 @@ typedef struct RetainConflictInfoData
TimestampTz reply_time; /* when the publisher responds with status */
TimestampTz last_recv_time; /* when the last message was received */
TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+ TimestampTz next_request_time; /* when to send next request */
int xid_advancement_interval; /* how much time (ms) to wait
* before attempting to advance
* the non-removable transaction
@@ -4071,11 +4072,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* - 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.
+ * do not proceed if:
+ * a) The publisher's clock lags behind that of the subscriber. 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.
+ * b) 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.
+ * until the publisher's clock catches up and all these transactions have
+ * completed.
*
* - RCI_WAIT_FOR_LOCAL_FLUSH:
* Advance the non-removable transaction ID if the current flush location has
@@ -4083,8 +4092,9 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* The overall state progression is: GET_CANDIDATE_XID ->
* REQUEST_PUBLISHER_STATUS -> WAIT_FOR_PUBLISHER_STATUS -> (loop to
- * REQUEST_PUBLISHER_STATUS if concurrent remote transactions persist) ->
- * WAIT_FOR_LOCAL_FLUSH -> loop back to GET_CANDIDATE_XID.
+ * REQUEST_PUBLISHER_STATUS if publisher's clock lags behind or if concurrent
+ * remote transactions persist) -> 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
@@ -4179,6 +4189,15 @@ get_candidate_xid(RetainConflictInfoData *data)
next_full_xid = ReadNextFullTransactionId();
epoch = EpochFromFullTransactionId(next_full_xid);
+ /*
+ * Get a new timestamp after obtaining the oldest running transaction ID.
+ * This ensures that all transactions preceding this transaction ID were
+ * assigned a commit timestamp earlier than the current timestamp. In
+ * wait_for_local_flush(), this timestamp is used to prevent the premature
+ * advancement of the oldest_nonremovable_xid due to clock skew.
+ */
+ data->candidate_xid_time = GetCurrentTimestamp();
+
/* Compute the epoch of the oldest_running_xid */
if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
epoch--;
@@ -4244,6 +4263,7 @@ static void
request_publisher_status(RetainConflictInfoData *data)
{
static StringInfo request_message = NULL;
+ TimestampTz now;
if (!request_message)
{
@@ -4255,6 +4275,12 @@ request_publisher_status(RetainConflictInfoData *data)
else
resetStringInfo(request_message);
+ now = GetCurrentTimestamp();
+
+ /* Return if it is not yet time to send the next request */
+ if (now < data->next_request_time)
+ return;
+
/*
* Send the current time to update the remote walsender's latest reply
* message received time.
@@ -4284,6 +4310,7 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
{
FullTransactionId remote_full_xid;
uint32 remote_epoch = data->remote_epoch;
+ TimestampTz skew;
/*
* Return if we have requested but not yet received the publisher status.
@@ -4291,6 +4318,21 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
if (!status_received)
return;
+ skew = data->candidate_xid_time - data->reply_time;
+
+ /*
+ * If the publisher's clock lags behind that of the subscriber, wait for
+ * the publisher's clock to catch up before proceeding to the next phase.
+ */
+ if (skew > 0)
+ {
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+ data->next_request_time = GetCurrentTimestamp() + skew;
+
+ maybe_advance_nonremovable_xid(data, false);
+ return;
+ }
+
if (!FullTransactionIdIsValid(data->last_phase_at))
data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
data->remote_nextxid);
@@ -4327,22 +4369,6 @@ 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
@@ -4398,6 +4424,7 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->last_phase_at = InvalidFullTransactionId;
data->candidate_xid = InvalidFullTransactionId;
data->reply_time = 0;
+ data->next_request_time = 0;
/* process the next phase */
maybe_advance_nonremovable_xid(data, false);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 8ae2783440..598d3b7374 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -2679,6 +2679,7 @@ ProcessStandbyPSRequestMessage(void)
FullTransactionId nextFullXid;
WalSnd *walsnd = MyWalSnd;
TimestampTz replyTime;
+ TimestampTz now;
/*
* This shouldn't happen because we don't support getting primary status
@@ -2697,6 +2698,14 @@ ProcessStandbyPSRequestMessage(void)
walsnd->replyTime = replyTime;
SpinLockRelease(&walsnd->mutex);
+ /*
+ * Get a new timestamp at the beginning to ensure that any transactions
+ * assigned a timestamp earlier than this point either complete before the
+ * current WAL (Write-Ahead Logging) write position or are detected in
+ * GetOldestTransactionIdInCommit().
+ */
+ now = GetCurrentTimestamp();
+
oldestXidInCommit = GetOldestTransactionIdInCommit();
nextFullXid = ReadNextFullTransactionId();
lsn = GetXLogWriteRecPtr();
@@ -2710,7 +2719,7 @@ ProcessStandbyPSRequestMessage(void)
pq_sendint32(&output_message, oldestXidInCommit);
pq_sendint32(&output_message, XidFromFullTransactionId(nextFullXid));
pq_sendint32(&output_message, EpochFromFullTransactionId(nextFullXid));
- pq_sendint64(&output_message, GetCurrentTimestamp());
+ pq_sendint64(&output_message, now);
/* ... and send it wrapped in CopyData */
pq_putmessage_noblock('d', output_message.data, output_message.len);
--
2.30.0.windows.2
v18-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v18-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 4af7e9d503a96ce10da675ba9c05db4266a34ff6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v18 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 defined by wal_receiver_status_interval
to minimize performance impact.
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/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 417 ++++++++++++++++++++-
src/backend/replication/walsender.c | 54 +++
src/backend/storage/ipc/procarray.c | 59 +++
src/include/replication/worker_internal.h | 18 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
10 files changed, 649 insertions(+), 4 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..49a4d3ab1b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 3ebd7c4041..611926c7d2 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 8b19642044..f8cf00c984 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 9e50c880f8..654078a726 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#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"
#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 +277,44 @@ 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 last_recv_time; /* when the last message was received */
+ TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+} RetainConflictInfoData;
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -339,6 +379,12 @@ 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. Refer to
+ * send_feedback() for details on its usage.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +425,15 @@ 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 inline bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3573,6 +3628,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
@@ -3651,6 +3707,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);
@@ -3677,6 +3735,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3758,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);
@@ -3706,6 +3796,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)
{
/*
@@ -3803,6 +3898,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
@@ -3838,7 +3935,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;
@@ -3916,6 +4012,325 @@ 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 if concurrent remote transactions persist) ->
+ * 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Compute the candidate_xid and request the publisher status at most once
+ * per wal_receiver_status_interval. This is to avoid using CPU and
+ * network resources without making much progress.
+ *
+ * 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.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ wal_receiver_status_interval * 1000))
+ return;
+
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ return;
+
+ 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * It is safe to add new tables with initial states to the subscription
+ * after this check because WAL positions of changes from these new
+ * tables, which will be applied, should be greater than remote_lsn and
+ * are included in transactions with later commit timestamps. So, there is
+ * no need to wait for these changes to be applied in this round of
+ * advancement.
+ */
+ if (!AllTablesyncsReady())
+ return;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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, now 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;
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
/*
* 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 371eef3ddd..68ca0c5712 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2314,6 +2316,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2660,6 +2666,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 c769b1aa3e..ae6da9fd91 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 9646261d7e..1eab8a5e46 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,24 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions are still non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ 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 5a3dd5d2d4..7eca49e883 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 56af0b40b3..c388eec6a5 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 e1c4f913f8..b5ea6bb5b8 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2474,6 +2474,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v18-0002-Dynamically-adjust-xid-advancement-interval.patchapplication/octet-stream; name=v18-0002-Dynamically-adjust-xid-advancement-interval.patchDownload
From 9aac9ebe19df7d960796db3ed96bf5b2fd4d02a0 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 16:37:00 +0800
Subject: [PATCH v18 2/7] Dynamically adjust xid advancement interval
If no new transaction ID has been assigned since the last advancement, the
interval is doubled. This increase is limited by the
wal_receiver_status_interval if it is not zero, or otherwise restricted to a
maximum of 3 minutes. If a new transaction ID is detected, the interval is
reset to a minimum of 100ms.
---
src/backend/replication/logical/worker.c | 62 +++++++++++++++++++++++-
1 file changed, 61 insertions(+), 1 deletion(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 654078a726..d5772a6b22 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -313,8 +313,19 @@ typedef struct RetainConflictInfoData
TimestampTz reply_time; /* when the publisher responds with status */
TimestampTz last_recv_time; /* when the last message was received */
TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+ int xid_advancement_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.
+ */
+#define MIN_XID_ADVANCEMENT_INTERVAL 100
+#define MAX_XID_ADVANCEMENT_INTERVAL 180000
+
/* errcontext tracker */
static ApplyErrorCallbackArg apply_error_callback_arg =
{
@@ -428,6 +439,8 @@ 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 adjust_xid_advancement_interval(RetainConflictInfoData *data,
+ bool new_xid_found);
static void request_publisher_status(RetainConflictInfoData *data);
static void wait_for_publisher_status(RetainConflictInfoData *data,
bool status_received);
@@ -3835,6 +3848,13 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to attempt advancing the
+ * non-removable transaction ID.
+ */
+ if (data.phase == RCI_GET_CANDIDATE_XID && data.xid_advancement_interval)
+ wait_time = Min(wait_time, data.xid_advancement_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -4129,7 +4149,7 @@ get_candidate_xid(RetainConflictInfoData *data)
* can consider the other interval or a separate GUC if the need arises.
*/
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
- wal_receiver_status_interval * 1000))
+ data->xid_advancement_interval))
return;
data->candidate_xid_time = now;
@@ -4147,7 +4167,12 @@ get_candidate_xid(RetainConflictInfoData *data)
/* Return if the oldest_nonremovable_xid cannot be advanced */
if (FullTransactionIdFollowsOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
full_xid))
+ {
+ adjust_xid_advancement_interval(data, false);
return;
+ }
+
+ adjust_xid_advancement_interval(data, true);
data->candidate_xid = full_xid;
data->phase = RCI_REQUEST_PUBLISHER_STATUS;
@@ -4156,6 +4181,41 @@ get_candidate_xid(RetainConflictInfoData *data)
maybe_advance_nonremovable_xid(data, false);
}
+/*
+ * Adjust the interval for advancing non-removable transaction IDs.
+ *
+ * If no new transaction ID has been assigned since the last advancement, the
+ * interval is doubled. This increase is limited by the
+ * wal_receiver_status_interval if it is not zero, or otherwise restricted to a
+ * maximum of 3 minutes. If a new transaction ID is detected, the interval is
+ * reset to a minimum of 100ms.
+ */
+static void
+adjust_xid_advancement_interval(RetainConflictInfoData *data, bool new_xid_found)
+{
+ if (!new_xid_found && data->xid_advancement_interval)
+ {
+ int max_interval = wal_receiver_status_interval
+ ? wal_receiver_status_interval * 1000
+ : MAX_XID_ADVANCEMENT_INTERVAL;
+
+ /*
+ * No new transaction ID assigned since the last check, so double the
+ * interval, but not beyond the maximum allowable value.
+ */
+ data->xid_advancement_interval = Min(data->xid_advancement_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_advancement_interval = MIN_XID_ADVANCEMENT_INTERVAL;
+ }
+}
+
/*
* Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
*/
--
2.30.0.windows.2
v18-0003-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v18-0003-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 774625317ff57a1871c4ca66c9605bb69a76d060 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v18 3/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.
---
src/backend/replication/logical/launcher.c | 210 +++++++++++++++++++++
src/backend/replication/slot.c | 13 ++
src/backend/replication/slotfuncs.c | 4 +
src/backend/replication/walsender.c | 2 +
src/include/replication/slot.h | 8 +
5 files changed, 237 insertions(+)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f8cf00c984..1a400d2228 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1119,6 +1128,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1159,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();
@@ -1167,14 +1181,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1250,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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,144 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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/slot.c b/src/backend/replication/slot.c
index 4a206f9527..937afa5b6a 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -288,6 +288,19 @@ ReplicationSlotValidateName(const char *name, int elevel)
return true;
}
+/*
+ * Report an error if the replication slot name is "pg_conflict_detection".
+ */
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
+}
+
/*
* Create a new replication slot and mark it as used by this backend.
*
diff --git a/src/backend/replication/slotfuncs.c b/src/backend/replication/slotfuncs.c
index 488a161b3e..d547d98124 100644
--- a/src/backend/replication/slotfuncs.c
+++ b/src/backend/replication/slotfuncs.c
@@ -38,6 +38,8 @@ create_physical_replication_slot(char *name, bool immediately_reserve,
{
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/* acquire replication slot, this will check for conflicting names */
ReplicationSlotCreate(name, false,
temporary ? RS_TEMPORARY : RS_PERSISTENT, false,
@@ -124,6 +126,8 @@ create_logical_replication_slot(char *name, char *plugin,
Assert(!MyReplicationSlot);
+ ErrorOnReservedSlotName(name);
+
/*
* Acquire a logical decoding slot, this will check for conflicting names.
* Initially create persistent slot as ephemeral - that allows us to
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 68ca0c5712..8ae2783440 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -1190,6 +1190,8 @@ CreateReplicationSlot(CreateReplicationSlotCmd *cmd)
parseCreateReplSlotOptions(cmd, &reserve_wal, &snapshot_action, &two_phase,
&failover);
+ ErrorOnReservedSlotName(cmd->slotname);
+
if (cmd->kind == REPLICATION_KIND_PHYSICAL)
{
ReplicationSlotCreate(cmd->slotname, false,
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index d2cf786fd5..82c4bf6160 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.
*
@@ -262,6 +269,7 @@ extern void ReplicationSlotMarkDirty(void);
/* misc stuff */
extern void ReplicationSlotInitialize(void);
extern bool ReplicationSlotValidateName(const char *name, int elevel);
+extern void ErrorOnReservedSlotName(const char *name);
extern void ReplicationSlotReserveWal(void);
extern void ReplicationSlotsComputeRequiredXmin(bool already_locked);
extern void ReplicationSlotsComputeRequiredLSN(void);
--
2.30.0.windows.2
v18-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v18-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 7a1a55d865acc1dcc468a14e57c399f1ea9b8073 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:49:03 +0800
Subject: [PATCH v18 4/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 | 11 ++
doc/src/sgml/monitoring.sgml | 11 ++
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 124 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++++
src/backend/replication/logical/worker.c | 51 ++++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +--
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 +-
11 files changed, 221 insertions(+), 34 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..55b8fb5ec6 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,17 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d0d176cc54..878e955e62 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index da9a8fe99f..adfea5fd6b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1370,6 +1370,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 68deea50f6..473da67a2d 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -312,7 +315,7 @@ retry:
*/
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -337,6 +340,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
@@ -412,7 +423,7 @@ retry:
/* 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;
@@ -463,6 +474,111 @@ retry:
return found;
}
+/*
+ * 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;
+
+ /* Return 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. */
+ 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);
+
+ 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 5d9ff626bd..ed743aa1a2 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 d5772a6b22..817da34c0b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2737,6 +2737,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin;
+ TransactionId localxid;
+ TimestampTz localts;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2753,15 +2756,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2772,7 +2771,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2791,19 +2790,27 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3125,7 +3132,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
bool found;
EPQState epqstate;
RepOriginId localorigin;
- TransactionId localxmin;
+ TransactionId localxid;
TimestampTz localts;
/* Get the matching local tuple from the partition. */
@@ -3135,17 +3142,25 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
InvalidOid, InvalidTransactionId,
InvalidRepOriginId, 0);
@@ -3157,7 +3172,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3168,7 +3183,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 6fc34f7494..cd714dfc4b 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2087,7 +2087,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2109,17 +2109,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 2dcc2d42da..6ffb1ccbdc 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 1c7fae0930..4179c79c3c 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"
@@ -670,7 +671,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 c759677ff5..93005c7267 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 3014d047fe..2f5d4c970c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,6 +2144,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,
@@ -2151,7 +2152,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.30.0.windows.2
v18-0005-Add-a-detect_update_deleted-option-to-subscripti.patchapplication/octet-stream; name=v18-0005-Add-a-detect_update_deleted-option-to-subscripti.patchDownload
From 3527039c0c36e13c5490beefb19d2da577da82d2 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 19 Dec 2024 12:23:10 +0800
Subject: [PATCH v18 5/7] Add a detect_update_deleted option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
option enabled.
Enabling detect_update_deleted is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/logical-replication.sgml | 7 +-
doc/src/sgml/ref/alter_subscription.sgml | 12 +-
doc/src/sgml/ref/create_subscription.sgml | 24 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/executor/execReplication.c | 11 +-
src/backend/replication/logical/launcher.c | 24 ++-
src/backend/replication/logical/worker.c | 26 +--
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 6 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
16 files changed, 368 insertions(+), 117 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cc6cf9bef0..cacbe7fa13 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8041,6 +8041,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subdetectupdatedeleted</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is enabled and the dead tuples on the subscriber that are still useful for
+ detecting this conflict are retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 55b8fb5ec6..bcb5df72c0 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1609,7 +1609,9 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
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
- <xref linkend="guc-track-commit-timestamp"/> is enabled.
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ are enabled.
</para>
</listitem>
</varlistentry>
@@ -2171,7 +2173,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <xref linkend="conflict-update-deleted"/> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..0bdf21a5ab 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-detect-update-deleted"><literal>detect_update_deleted</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain dead tuples will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..ace63d9389 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -435,6 +435,30 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained, and an additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent dead tuples from being removed.
+ </para>
+
+ <para>
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..c14a8dc474 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->detectupdatedeleted = subform->subdetectupdatedeleted;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index adfea5fd6b..710564eb85 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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)
+ subdetectupdatedeleted, 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 03e97730e7..ac260c1e9f 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_DETECT_UPDATE_DELETED 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 detectupdatedeleted;
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_remote_recovery(WalReceiverConn *wrconn,
+ bool detect_update_deleted);
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_DETECT_UPDATE_DELETED))
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED) &&
+ strcmp(defel->defname, "detect_update_deleted") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_DETECT_UPDATE_DELETED))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_DETECT_UPDATE_DELETED;
+ opts->detectupdatedeleted = 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_DETECT_UPDATE_DELETED | 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.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
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_remote_recovery(wrconn, opts.detectupdatedeleted);
+
/*
* 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 query_remote_recovery = 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_DETECT_UPDATE_DELETED | 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_DETECT_UPDATE_DELETED))
+ {
+ values[Anum_pg_subscription_subdetectupdatedeleted - 1] =
+ BoolGetDatum(opts.detectupdatedeleted);
+ replaces[Anum_pg_subscription_subdetectupdatedeleted - 1] = true;
+
+ if (opts.detectupdatedeleted && !track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 detect_update_deleted=true, meaning the
+ * remote server's recovery status may not be checked.
+ * Ensure this check is conducted now.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
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 is not in recovery if
+ * detect_update_deleted is enabled.
+ */
+ query_remote_recovery = sub->detectupdatedeleted;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the detect_update_deleted setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(WalReceiverConn *wrconn, bool detect_update_deleted)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (!detect_update_deleted)
+ return;
+
+ 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 detect_update_deleted if the publisher is in recovery."));
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 473da67a2d..ddd61fd150 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -507,14 +507,17 @@ FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ /* Error out if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "track_commit_timestamp"));
+
*delete_xid = InvalidTransactionId;
*delete_origin = InvalidRepOriginId;
*delete_time = 0;
- /* Return if the commit timestamp data is not available */
- if (!track_commit_timestamp)
- return false;
-
/* Get the cutoff xmin for HeapTupleSatisfiesVacuum */
oldestXmin = GetOldestNonRemovableTransactionId(rel);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 1a400d2228..c713623789 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -157,6 +157,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->detectupdatedeleted = subform->subdetectupdatedeleted;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1160,6 +1161,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1180,12 +1182,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * If any subscription enables detection of update_deleted
+ * conflicts, create a replication slot to retain dead tuples.
+ * Only advance xmin when all such subscriptions are enabled.
+ */
+ if (sub->detectupdatedeleted)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1194,10 +1204,11 @@ ApplyLauncherMain(Datum main_arg)
{
/*
* Collect non-removable transaction IDs from all apply
- * workers to determine the xmin for advancing the replication
+ * workers that detects update_deleted conflicts. This
+ * determines the new xmin for advancing the replication
* slot used in conflict detection.
*/
- if (can_advance_xmin)
+ if (sub->detectupdatedeleted && can_advance_xmin)
{
FullTransactionId nonremovable_xid;
@@ -1255,7 +1266,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1277,6 +1288,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 817da34c0b..895aa4b9bc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2737,9 +2737,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2793,7 +2793,8 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3131,9 +3132,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxid;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3145,7 +3146,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
- if (FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ if (MySubscription->detectupdatedeleted &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
&localxid, &localorigin,
&localts) &&
localorigin != replorigin_session_origin)
@@ -3162,8 +3164,8 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
*/
ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -4112,6 +4114,10 @@ static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received)
{
+ /* Exit early if retaining dead tuples is not required */
+ if (!MySubscription->detectupdatedeleted)
+ return;
+
/*
* It is sufficient to manage non-removable transaction ID for a
* subscription by the main apply worker to detect update_deleted conflict
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 19969e400f..9a52eb569c 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subdetectupdatedeleted;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subdetectupdatedeleted\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subdetectupdatedeleted\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subdetectupdatedeleted = PQfnumber(res, "subdetectupdatedeleted");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5009,6 +5017,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].subdetectupdatedeleted =
+ (strcmp(PQgetvalue(res, i, i_subdetectupdatedeleted), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5267,6 +5277,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subdetectupdatedeleted)
+ appendPQExpBufferStr(query, ", detect_update_deleted = 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 9c5ddd20cf..1813557a7a 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subdetectupdatedeleted;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 2657abdc72..2f8b911574 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ", subdetectupdatedeleted AS \"%s\"\n",
+ gettext_noop("Detect update deleted"));
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index beaff6578a..a811b4a7d1 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 subdetectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
+
#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 detectupdatedeleted; /* True if the detection of
+ * update_deleted conflict is enabled */
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 1443e1d929..bbad3f1b89 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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 - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+ERROR: detect_update_deleted requires a Boolean value
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..6f271836cc 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - detect_update_deleted must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = foo);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, detect_update_deleted = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (detect_update_deleted = true);
+
+\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;
--
2.30.0.windows.2
v18-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchapplication/octet-stream; name=v18-0006-Add-a-tap-test-to-verify-the-new-slot-xmin-mecha.patchDownload
From 766d500efdb2b922181be232e04421dff8f4d5ce Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Dec 2024 15:51:23 +0800
Subject: [PATCH v18 6/7] Add a tap test to verify the new slot xmin mechanism
---
src/test/subscription/meson.build | 1 +
.../t/035_confl_update_deleted.pl | 169 ++++++++++++++++++
2 files changed, 170 insertions(+)
create mode 100644 src/test/subscription/t/035_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index b2395e7b57..a80c7f1656 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_confl_update_deleted.pl b/src/test/subscription/t/035_confl_update_deleted.pl
new file mode 100644
index 0000000000..85da557e91
--- /dev/null
+++ b/src/test/subscription/t/035_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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, detect_update_deleted = 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, detect_update_deleted = 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+my $logfile = slurp_file($node_B->logfile(), $log_location);
+ok( $logfile =~
+ qr/conflict detected on relation "public.tab": conflict=delete_origin_differs.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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 the subscriptions that have the detect_update_deleted option enabled.
+###############################################################################
+
+$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();
--
2.30.0.windows.2
On Thu, Dec 19, 2024 at 4:34 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Sunday, December 15, 2024 9:39 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
5. The apply worker needs to at least twice get the publisher status message to
advance oldest_nonremovable_xid once. It then uses the remote_lsn of the last
such message to ensure that it has been applied locally. Such a remote_lsn
could be a much later value than required leading to delay in advancing
oldest_nonremovable_xid. How about if while first time processing the
publisher_status message on walsender, we get the
latest_transaction_in_commit by having a function
GetLatestTransactionIdInCommit() instead of
GetOldestTransactionIdInCommit() and then simply wait till that proc has
written commit WAL (aka wait till it clears DELAY_CHKPT_IN_COMMIT)?
Then get the latest LSN wrote and send that to apply worker waiting for the
publisher_status message. If this is feasible then we should be able to
advance oldest_nonremovable_xid with just one publisher_status message.
Won't that be an improvement over current? If so, we can even further try to
improve it by just using commit_LSN of the transaction returned by
GetLatestTransactionIdInCommit(). One idea is that we can try to use
MyProc->waitLSN which we are using in synchronous replication for our
purpose. See SyncRepWaitForLSN.I will do more performance tests on this and address if it improves
the performance.
Did you check this idea? Again, thinking about this, I see a downside
to the new proposal. In the new proposal, the walsender needs to
somehow wait for the transactions in the commit which essentially
means that it may lead delay in decoding and sending the decoded WAL.
But it is still worth checking the impact of such a change, if nothing
else, we can add a short comment in the code to suggest such an
improvement is not worthwhile.
--
With Regards,
Amit Kapila.
On Wed, Dec 25, 2024 at 8:13 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the new version patch set which addressed all other comments.
Review comments on 0001 and 0002
=============================
1.
/*
+ * 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;
There is no comment in the data-structure RetainConflictInfoData that
indicates the fields used to determine the timing for the next round
of transaction ID advancement. Can we add a comment in
RetainConflictInfoData to indicate the same?
2.
+ int xid_advancement_interval; /* how much time (ms) to wait
+ * before attempting to advance
+ * the non-removable transaction
+ * ID */
} RetainConflictInfoData;
Shall we rename it to a bit simpler name xid_advance_interval? If you
agree with this change, we can probably rename
adjust_xid_advancement_interval() to adjust_xid_advance_interval() as
well.
3.
+/*
+ * The minimum (100ms) and maximum (3 minutes) intervals for advancing
+ * non-removable transaction IDs.
+ */
+#define MIN_XID_ADVANCEMENT_INTERVAL 100
+#define MAX_XID_ADVANCEMENT_INTERVAL 180000
Is there any reason to keep the maximum value as 3 minutes? If not
then mention that it is arbitrary and sufficient to not cause any
undue network traffic.
4.
@@ -4129,7 +4149,7 @@ get_candidate_xid(RetainConflictInfoData *data)
* can consider the other interval or a separate GUC if the need arises.
*/
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
- wal_receiver_status_interval * 1000))
+ data->xid_advancement_interval))
The comment atop the above change in the second patch needs to change.
5.
+static void
+adjust_xid_advancement_interval(RetainConflictInfoData *data, bool
new_xid_found)
Let's move the location of this function to after
can_advance_nonremovable_xid(). This is to keep the functions to
transition the retain_data_phases together.
Apart from the above, I have made changes in a few comments in the
attached. Please include those after review and combine 0001 and 0002
as one patch.
--
With Regards,
Amit Kapila.
Attachments:
v18_amit_1.patch.txttext/plain; charset=US-ASCII; name=v18_amit_1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d5772a6b22..090aae126b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -391,8 +391,9 @@ static XLogRecPtr skip_xact_finish_lsn = InvalidXLogRecPtr;
static BufFile *stream_fd = NULL;
/*
- * The remote WAL position that has been applied and flushed locally. Refer to
- * send_feedback() for details on its usage.
+ * 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;
@@ -3849,7 +3850,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
wait_time = NAPTIME_PER_CYCLE;
/*
- * Ensure to wake up when it's possible to attempt advancing the
+ * 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_advancement_interval)
@@ -4066,7 +4067,7 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* The overall state progression is: GET_CANDIDATE_XID ->
* REQUEST_PUBLISHER_STATUS -> WAIT_FOR_PUBLISHER_STATUS -> (loop to
- * REQUEST_PUBLISHER_STATUS if concurrent remote transactions persist) ->
+ * 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
@@ -4184,11 +4185,14 @@ get_candidate_xid(RetainConflictInfoData *data)
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
- * If no new transaction ID has been assigned since the last advancement, the
- * interval is doubled. This increase is limited by the
- * wal_receiver_status_interval if it is not zero, or otherwise restricted to a
- * maximum of 3 minutes. If a new transaction ID is detected, the interval is
- * reset to a minimum of 100ms.
+ * 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.
*/
static void
adjust_xid_advancement_interval(RetainConflictInfoData *data, bool new_xid_found)
@@ -4200,8 +4204,8 @@ adjust_xid_advancement_interval(RetainConflictInfoData *data, bool new_xid_found
: MAX_XID_ADVANCEMENT_INTERVAL;
/*
- * No new transaction ID assigned since the last check, so double the
- * interval, but not beyond the maximum allowable value.
+ * No new transaction ID has been assigned since the last check, so
+ * double the interval, but not beyond the maximum allowable value.
*/
data->xid_advancement_interval = Min(data->xid_advancement_interval * 2,
max_interval);
@@ -4331,11 +4335,8 @@ wait_for_local_flush(RetainConflictInfoData *data)
* effort.
*
* It is safe to add new tables with initial states to the subscription
- * after this check because WAL positions of changes from these new
- * tables, which will be applied, should be greater than remote_lsn and
- * are included in transactions with later commit timestamps. So, there is
- * no need to wait for these changes to be applied in this round of
- * advancement.
+ * after this check because any changes applied to these tables should have
+ * a WAL position greater than the data->remote_lsn.
*/
if (!AllTablesyncsReady())
return;
@@ -4354,8 +4355,7 @@ wait_for_local_flush(RetainConflictInfoData *data)
/*
* 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, now we can advance the non-removable transaction
- * ID.
+ * flushed locally. So, we can advance the non-removable transaction ID.
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 1eab8a5e46..22cdd0a591 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,20 +87,18 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions are still non-removable
- * to allow for the detection of update_deleted conflicts when applying
+ * 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.
*
* Note that this info cannot directly protect dead tuples from being
* prematurely frozen or removed. The logical replication launcher
* asynchronously collects this info to determine whether to advance the
- * xmin value of the replication slot.
+ * xmin value of its replication slot.
*
- * Therefore, FullTransactionId that includes both the transaction ID and
- * its epoch is used here instead of a single Transaction ID. This is
- * critical because without considering the epoch, the transaction ID
- * alone may appear as if it is in the future due to transaction ID
- * wraparound.
+ * We need to use FullTransactionId here because without considering the
+ * epoch, the transaction ID alone may appear as if it is in the future due
+ * to the transaction ID wraparound.
*/
FullTransactionId oldest_nonremovable_xid;
On Wed, Dec 25, 2024 at 8:13 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.
But why would it prevent the launcher from creating the slot? I think
we should add this check in the function
ReplicationSlotValidateName(). Another related point:
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name));
Won't it be sufficient to check using an existing IsReservedName()?
Even, if not, then also we should keep that as part of the check
similar to what we are doing in pg_replication_origin_create().
03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.
Sounds reasonable but OTOH, all other places that create physical
slots (which we are doing here) don't use this trick. So, don't they
need similar reliability? Also, add some comments as to why we are
initially creating the RS_EPHEMERAL slot as we have at other places.
Few other comments on 0003
=======================
1.
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
How will it help to try advancing slot_xmin when xmin is invalid?
2.
@@ -1167,14 +1181,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ can_advance_xmin = false;
In ApplyLauncherMain(), if one of the subscriptions is disabled (say
the last one in sublist), then can_advance_xmin will become false in
the above code. Now, later, as quoted in comment-1, the patch
overrides xmin to InvalidFullTransactionId if can_advance_xmin is
false. Won't that lead to the wrong computation of xmin?
3.
+ 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;
Can't we use MyReplicationSlot instead of introducing a new boolean
slot_maybe_exist?
In any case, how does the above code deal with the case where the
launcher is restarted for some reason and there is no subscription
after that? Will it be possible to drop the slot in that case?
--
With Regards,
Amit Kapila.
On Wed, 25 Dec 2024 at 08:13, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.05. check_remote_recovery()
Can we add a test case related with this?
I think the code path is already tested, and I am a bit unsure if we want to setup
a standby to test the ERROR case, so didn't add this.---
Attach the new version patch set which addressed all other comments.
Based on some off-list discussions with Sawada-san and Amit, it would be better
if the apply worker can avoid reporting an ERROR if the publisher's clock's
lags behind that of the subscriber, so I implemented a new 0007 patch to allow
the apply worker to wait for the clock skew to pass and then send a new request
to the publisher for the latest status. The implementation is as follows:Since we have the time (reply_time) on the walsender when it confirms that all
the committing transactions have finished, it means any subsequent transactions
on the publisher should be assigned a commit timestamp later then reply_time.
And the (candidate_xid_time) when it determines the oldest active xid. Any old
transactions on the publisher that have finished should have a commit timestamp
earlier than the candidate_xid_time.The apply worker can compare the candidate_xid_time with reply_time. If
candidate_xid_time is less than the reply_time, then it's OK to advance the xid
immdidately. If candidate_xid_time is greater than reply_time, it means the
clock of publisher is behind that of the subscriber, so the apply worker can
wait for the skew to pass before advancing the xid.Since this is considered as an improvement, we can focus on this after
pushing the main patches.
Conflict detection of truncated updates is detected as update_missing
and deleted update is detected as update_deleted. I was not sure if
truncated updates should also be detected as update_deleted, as the
document says truncate operation is "It has the same effect as an
unqualified DELETE on each table" at [1]https://www.postgresql.org/docs/devel/sql-truncate.html.
I tried with the following three node(N1,N2 & N3) setup with
subscriber on N3 subscribing to the publisher pub1 in N1 and publisher
pub2 in N2:
N1 - pub1
N2 - pub2
N3 - sub1 -> pub1(N1) and sub2 -> pub2(N2)
-- Insert a record in N1
insert into t1 values(1);
-- Insert a record in N2
insert into t1 values(1);
-- Now N3 has the above inserts from N1 and N2
N3=# select * from t1;
c1
----
1
1
(2 rows)
-- Truncate t1 from N2
N2=# truncate t1;
TRUNCATE TABLE
-- Now N3 has no records:
N3=# select * from t1;
c1
----
(0 rows)
-- Update from N1 to generated a conflict
postgres=# update t1 set c1 = 2;
UPDATE 1
N1=# select * from t1;
c1
----
2
(1 row)
--- N3 logs the conflict as update_missing
2025-01-02 12:21:37.388 IST [24803] LOG: conflict detected on
relation "public.t1": conflict=update_missing
2025-01-02 12:21:37.388 IST [24803] DETAIL: Could not find the row to
be updated.
Remote tuple (2); replica identity full (1).
2025-01-02 12:21:37.388 IST [24803] CONTEXT: processing remote data
for replication origin "pg_16387" during message type "UPDATE" for
replication target relation "public.t1" in transaction 757, finished
at 0/17478D0
-- Insert a record with value 2 in N2
N2=# insert into t1 values(2);
INSERT 0 1
-- Now N3 has the above inserted records:
N3=# select * from t1;
c1
----
2
(1 row)
-- Delete this record from N2:
N2=# delete from t1;
DELETE 1
-- Now N3 has no records:
N3=# select * from t1;
c1
----
(0 rows)
-- Update from N1 to generate a conflict
postgres=# update t1 set c1 = 3;
UPDATE 1
--- N3 logs the conflict as update_deleted
2025-01-02 12:22:38.036 IST [24803] LOG: conflict detected on
relation "public.t1": conflict=update_deleted
2025-01-02 12:22:38.036 IST [24803] DETAIL: The row to be updated was
deleted by a different origin "pg_16388" in transaction 764 at
2025-01-02 12:22:29.025347+05:30.
Remote tuple (3); replica identity full (2).
2025-01-02 12:22:38.036 IST [24803] CONTEXT: processing remote data
for replication origin "pg_16387" during message type "UPDATE" for
replication target relation "public.t1" in transaction 758, finished
at 0/174D240
I'm not sure if this behavior is expected or not. If this is expected
can we mention this in the documentation for the user to handle the
conflict resolution accordingly in these cases.
Thoughts?
[1]: https://www.postgresql.org/docs/devel/sql-truncate.html
Regards,
Vignesh
On Wed, 25 Dec 2024 at 08:13, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.05. check_remote_recovery()
Can we add a test case related with this?
I think the code path is already tested, and I am a bit unsure if we want to setup
a standby to test the ERROR case, so didn't add this.---
Attach the new version patch set which addressed all other comments.
Few suggestions:
1) If we have a subscription with detect_update_deleted option and we
try to upgrade it with default settings(in case dba forgot to set
track_commit_timestamp), the upgrade will fail after doing a lot of
steps like that mentioned in ok below:
Setting locale and encoding for new cluster ok
Analyzing all rows in the new cluster ok
Freezing all rows in the new cluster ok
Deleting files from new pg_xact ok
Copying old pg_xact to new server ok
Setting oldest XID for new cluster ok
Setting next transaction ID and epoch for new cluster ok
Deleting files from new pg_multixact/offsets ok
Copying old pg_multixact/offsets to new server ok
Deleting files from new pg_multixact/members ok
Copying old pg_multixact/members to new server ok
Setting next multixact ID and offset for new cluster ok
Resetting WAL archives ok
Setting frozenxid and minmxid counters in new cluster ok
Restoring global objects in the new cluster ok
Restoring database schemas in the new cluster
postgres
*failure*
We should detect this at an earlier point somewhere like in
check_new_cluster_subscription_configuration and throw an error from
there.
2) Also should we include an additional slot for the
pg_conflict_detection slot while checking max_replication_slots.
Though this error will occur after the upgrade is completed, it may be
better to include the slot during upgrade itself so that the DBA need
not handle this error separately after the upgrade is completed.
3) We have reserved the pg_conflict_detection name in this version, so
if there was a replication slot with the name pg_conflict_detection in
the older version, the upgrade will fail at a very later stage like an
earlier upgrade shown. I feel we should check if the old cluster has
any slot with the name pg_conflict_detection and throw an error
earlier itself:
+void
+ErrorOnReservedSlotName(const char *name)
+{
+ if (strcmp(name, CONFLICT_DETECTION_SLOT) == 0)
+ ereport(ERROR,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\"
is reserved",
+ name));
+}
4) We should also mention something like below in the documentation so
the user can be aware of it:
The slot name cannot be created with pg_conflict_detection, as this is
reserved for logical replication conflict detection.
Regards,
Vignesh
On Wed, Dec 25, 2024 at 8:13 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the new version patch set which addressed all other comments.
Some more miscellaneous comments:
=============================
1.
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
The comments related to this change should be updated in EndPrepare()
and RecordTransactionCommitPrepared(). They still refer to the
DELAY_CHKPT_START flag. We should update the comments explaining why a
similar change is not required for prepare or commit_prepare, if there
is one.
2.
static bool
tuples_equal(TupleTableSlot *slot1, TupleTableSlot *slot2,
- TypeCacheEntry **eq)
+ TypeCacheEntry **eq, Bitmapset *columns)
{
int attrnum;
@@ -337,6 +340,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;
Update the comment atop tuples_equal to reflect this change.
3.
+FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
...
...
+ /* 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);
+
+ if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) ==
HEAPTUPLE_RECENTLY_DEAD)
+ dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!dead)
+ continue;
Why do we need to check only for HEAPTUPLE_RECENTLY_DEAD and not
HEAPTUPLE_DEAD? IIUC, we came here because we couldn't find the live
tuple, now whether the tuple is DEAD or RECENTLY_DEAD, why should it
matter to detect update_delete conflict?
4. In FindMostRecentlyDeletedTupleInfo(), add comments to state why we
need to use SnapshotAny.
5.
+
+ <varlistentry
id="sql-createsubscription-params-with-detect-update-deleted">
+ <term><literal>detect_update_deleted</literal>
(<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the detection of <xref
linkend="conflict-update-deleted"/>
+ is enabled. The default is <literal>false</literal>. If set to
+ true, the dead tuples on the subscriber that are still useful for
+ detecting <xref linkend="conflict-update-deleted"/>
+ are retained,
One of the purposes of retaining dead tuples is to detect
update_delete conflict. But, I also see the following in 0001's commit
message: "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."
which indicates there are other cases where retaining dead tuples can
help. So, I was thinking about whether to name this new option as
retain_dead_tuples or something along those lines?
BTW, it is not clear how retaining dead tuples will help the detection
update_origin_differs. Will it happen when the tuple is inserted or
updated on the subscriber and then when we try to update the same
tuple due to remote update, the commit_ts information of the xact is
not available because the same is already removed by vacuum? This
should happen for the update case for the new row generated by the
update operation as that will be used in comparison. Can you please
show it be a test case even if it is manual?
Can't it happen for delete_origin_differs as well for the same reason?
6. I feel we should keep 0004 as a later patch. We can ideally
consider committing 0001, 0002, 0003, 0005, and 0006 (or part of 0006
to get some tests that are relevant) as one unit and then the patch to
detect and report update_delete conflict. What do you think?
--
With Regards,
Amit Kapila.
On Tue, Dec 24, 2024 at 6:43 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.05. check_remote_recovery()
Can we add a test case related with this?
I think the code path is already tested, and I am a bit unsure if we want to setup
a standby to test the ERROR case, so didn't add this.---
Attach the new version patch set which addressed all other comments.
Based on some off-list discussions with Sawada-san and Amit, it would be better
if the apply worker can avoid reporting an ERROR if the publisher's clock's
lags behind that of the subscriber, so I implemented a new 0007 patch to allow
the apply worker to wait for the clock skew to pass and then send a new request
to the publisher for the latest status. The implementation is as follows:Since we have the time (reply_time) on the walsender when it confirms that all
the committing transactions have finished, it means any subsequent transactions
on the publisher should be assigned a commit timestamp later then reply_time.
And the (candidate_xid_time) when it determines the oldest active xid. Any old
transactions on the publisher that have finished should have a commit timestamp
earlier than the candidate_xid_time.The apply worker can compare the candidate_xid_time with reply_time. If
candidate_xid_time is less than the reply_time, then it's OK to advance the xid
immdidately. If candidate_xid_time is greater than reply_time, it means the
clock of publisher is behind that of the subscriber, so the apply worker can
wait for the skew to pass before advancing the xid.Since this is considered as an improvement, we can focus on this after
pushing the main patches.
Thank you for updating the patches!
I have one comment on the 0001 patch:
+ /*
+ * The changes made by this and later transactions are still
non-removable
+ * to allow for the detection of update_deleted conflicts when applying
+ * changes in this logical replication worker.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of the replication slot.
+ *
+ * Therefore, FullTransactionId that includes both the
transaction ID and
+ * its epoch is used here instead of a single Transaction ID. This is
+ * critical because without considering the epoch, the transaction ID
+ * alone may appear as if it is in the future due to transaction ID
+ * wraparound.
+ */
+ FullTransactionId oldest_nonremovable_xid;
The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID
wraparound happens. But once we set the oldest-nonremovable-xid it
prevents XIDs from being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wed, 25 Dec 2024 at 08:13, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.05. check_remote_recovery()
Can we add a test case related with this?
I think the code path is already tested, and I am a bit unsure if we want to setup
a standby to test the ERROR case, so didn't add this.---
Attach the new version patch set which addressed all other comments.
Few comments:
1) In case there are no logical replication workers, the launcher
process just logs a warning "out of logical replication worker slots"
and continues. Whereas in case of "pg_conflict_detection" replication
slot creation launcher throw an error and the launcher exits, can we
throw a warning in this case too:
2025-01-02 10:24:41.899 IST [4280] ERROR: all replication slots are in use
2025-01-02 10:24:41.899 IST [4280] HINT: Free one or increase
"max_replication_slots".
2025-01-02 10:24:42.148 IST [4272] LOG: background worker "logical
replication launcher" (PID 4280) exited with exit code 1
2) Currently, we do not detect when the track_commit_timestamp setting
is disabled for a subscription immediately after the worker starts.
Instead, it is detected later during conflict detection. Since
changing the track_commit_timestamp GUC requires a server restart, it
would be beneficial for DBAs if the error were raised immediately.
This way, DBAs would be aware of the issue as they monitor the server
restart and can take the necessary corrective actions without delay.
3) Tab completion missing for CREATE SUBSCRIPTION does not include
detect_update_deleted option:
postgres=# create subscription sub3 CONNECTION 'dbname=postgres
host=localhost port=5432' publication pub1 with (
BINARY COPY_DATA DISABLE_ON_ERROR FAILOVER
PASSWORD_REQUIRED SLOT_NAME SYNCHRONOUS_COMMIT
CONNECT CREATE_SLOT ENABLED ORIGIN
RUN_AS_OWNER STREAMING TWO_PHASE
4) Tab completion missing for ALTER SUBSCRIPTION does not include
detect_update_deleted option:
ALTER SUBSCRIPTION sub3 SET (
BINARY FAILOVER PASSWORD_REQUIRED SLOT_NAME
SYNCHRONOUS_COMMIT
DISABLE_ON_ERROR ORIGIN RUN_AS_OWNER STREAMING
TWO_PHASE
5) Copyright year can be updated to 2025:
+++ b/src/test/subscription/t/035_confl_update_deleted.pl
@@ -0,0 +1,169 @@
+
+# Copyright (c) 2024, PostgreSQL Global Development Group
+
+# Test the CREATE SUBSCRIPTION 'detect_update_deleted' parameter and its
+# interaction with the xmin value of replication slots.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
6) This include is not required, I was able to compile without it:
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,12 +173,14 @@
#include "replication/logicalrelation.h"
#include "replication/logicalworker.h"
#include "replication/origin.h"
+#include "replication/slot.h"
#include "replication/walreceiver.h"
Regards,
Vignesh
On Fri, Jan 3, 2025 at 12:06 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I have one comment on the 0001 patch:
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples from being + * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transaction ID + * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID
wraparound happens. But once we set the oldest-nonremovable-xid it
prevents XIDs from being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.
I also think that the slot's non-removal-xid should ensure that we
never allow xid to advance to a level where it can cause a wraparound
for the oldest-nonremovable-xid value stored in each worker because
the slot's value is the minimum of all workers. Now, if both of us are
missing something then it is probably better to write some more
detailed comments as to how this can happen.
Along the same lines, I was thinking whether
RetainConflictInfoData->last_phase_at should be FullTransactionId but
I think that is correct because we can't stop wraparound from
happening on remote_node, right?
--
With Regards,
Amit Kapila.
On Fri, Jan 3, 2025 at 2:34 PM vignesh C <vignesh21@gmail.com> wrote:
Few comments:
1) In case there are no logical replication workers, the launcher
process just logs a warning "out of logical replication worker slots"
and continues. Whereas in case of "pg_conflict_detection" replication
slot creation launcher throw an error and the launcher exits, can we
throw a warning in this case too:
2025-01-02 10:24:41.899 IST [4280] ERROR: all replication slots are in use
2025-01-02 10:24:41.899 IST [4280] HINT: Free one or increase
"max_replication_slots".
2025-01-02 10:24:42.148 IST [4272] LOG: background worker "logical
replication launcher" (PID 4280) exited with exit code 1
This case is not the same because if give just WARNING and allow to
proceed then we won't be able to protect dead rows from removal. We
don't want the apply workers to keep working and making progress till
this slot is created. Am, I missing something? If not, we probably
need to ensure this if not already ensured. Also, we should mention in
the docs that the 'max_replication_slots' setting should consider this
additional slot.
2) Currently, we do not detect when the track_commit_timestamp setting
is disabled for a subscription immediately after the worker starts.
Instead, it is detected later during conflict detection.
I am not sure if an ERROR is required in the first place. Shouldn't we
simply not detect the update_delete in that case? It should be
documented that to detect this conflict 'track_commit_timestamp'
should be enabled. Don't we do the same thing for *_origin_differs
type of conflicts?
--
With Regards,
Amit Kapila.
On Thu, Jan 2, 2025 at 2:57 PM vignesh C <vignesh21@gmail.com> wrote:
Conflict detection of truncated updates is detected as update_missing
and deleted update is detected as update_deleted. I was not sure if
truncated updates should also be detected as update_deleted, as the
document says truncate operation is "It has the same effect as an
unqualified DELETE on each table" at [1].
This is expected behavior because TRUNCATE would immediately reclaim
space and remove all the data. So, there is no way to retain the
removed row.
--
With Regards,
Amit Kapila.
On Fri, Dec 20, 2024 at 12:41 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
In the test scenarios already shared on -hackers [1], where pgbench was run only on the publisher node in a pub-sub setup, no performance degradation was observed on either node.
In contrast, when pgbench was run only on the subscriber side with detect_update_deleted=on [2], the TPS performance was reduced due to dead tuple accumulation. This performance drop depended on the wal_receiver_status_interval—larger intervals resulted in more dead tuple accumulation on the subscriber node. However, after the improvement in patch v16-0002, which dynamically tunes the status request, the default TPS reduction was limited to only 1%.
We performed more benchmarks with the v16-patches where pgbench was run on both the publisher and subscriber, focusing on TPS performance. To summarize the key observations:
- No performance impact on the publisher as dead tuple accumulation does not occur on the publisher.
- The performance is reduced on the subscriber side (TPS reduction (~50%) [3] ) due to dead tuple retention for the conflict detection when detect_update_deleted=on.
- Performance reduction happens only on the subscriber side, as workload on the publisher is pretty high and the apply workers must wait for the amount of transactions with earlier timestamps to be applied and flushed before advancing the non-removable XID to remove dead tuples.
- To validate this further, we modified the patch to check only each transaction's commit_time and advance the non-removable XID if the commit_time is greater than candidate_xid_time. The benchmark results[4] remained consistent, showing similar performance reduction. This confirms that the performance impact on the subscriber side is a reasonable behavior if we want to detect the update_deleted conflict reliably.
We have also tested similar scenarios in physical streaming replication, to see the effect of enabling the hot_standby_feedback and recovery_min_apply_delay. The benchmark results[5] showed performance reduction in these cases as well, though impact was less compared to the update_deleted scenario because the physical walreceiver does not need to wait for specified WAL to be applied before sending the hot standby feedback message. However, as the recovery_min_apply_delay increased, a similar TPS reduction (~50%) was observed, aligning with the behavior seen in the update_deleted case.
The first impression after seeing such a performance dip will be not
to use such a setting but as the primary reason is that one
purposefully wants to retain dead tuples both in physical replication
and pub-sub environment, it is an expected outcome. Now, it is
possible that in real world people may not use exactly the setup we
have used to check the worst-case performance. For example, for a
pub-sub setup, one could imagine that writes happen on two nodes N1,
and N2 (both will be publisher nodes) and then all the changes from
both nodes will be assembled in the third node N3 (a subscriber node).
Or, the subscriber node, may not be set up for aggressive writes, Or,
one would be okay not to detect update_delete conflicts with complete
accuracy.
Based on the above, I think the performance reduction observed with the update_deleted patch is expected and necessary because the patch's main goal is to retain dead tuples for reliable conflict detection. Reducing this retention period would compromise the accuracy of update_deleted detection.
The point related to dead tuple accumulation (or database bloat) with
this setting should be documented similarly to what we document for
hot_standby_feedback. See hot_standby_feedback description in docs
[1]: https://www.postgresql.org/docs/devel/runtime-config-replication.html#RUNTIME-CONFIG-REPLICATION-STANDBY
--
With Regards,
Amit Kapila.
On Wed, Jan 1, 2025 at 11:06 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Dec 19, 2024 at 4:34 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Sunday, December 15, 2024 9:39 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
5. The apply worker needs to at least twice get the publisher status message to
advance oldest_nonremovable_xid once. It then uses the remote_lsn of the last
such message to ensure that it has been applied locally. Such a remote_lsn
could be a much later value than required leading to delay in advancing
oldest_nonremovable_xid. How about if while first time processing the
publisher_status message on walsender, we get the
latest_transaction_in_commit by having a function
GetLatestTransactionIdInCommit() instead of
GetOldestTransactionIdInCommit() and then simply wait till that proc has
written commit WAL (aka wait till it clears DELAY_CHKPT_IN_COMMIT)?
Then get the latest LSN wrote and send that to apply worker waiting for the
publisher_status message. If this is feasible then we should be able to
advance oldest_nonremovable_xid with just one publisher_status message.
Won't that be an improvement over current? If so, we can even further try to
improve it by just using commit_LSN of the transaction returned by
GetLatestTransactionIdInCommit(). One idea is that we can try to use
MyProc->waitLSN which we are using in synchronous replication for our
purpose. See SyncRepWaitForLSN.I will do more performance tests on this and address if it improves
the performance.Did you check this idea? Again, thinking about this, I see a downside
to the new proposal. In the new proposal, the walsender needs to
somehow wait for the transactions in the commit which essentially
means that it may lead delay in decoding and sending the decoded WAL.
But it is still worth checking the impact of such a change, if nothing
else, we can add a short comment in the code to suggest such an
improvement is not worthwhile.
Here are the performance test results for the proposed idea where the
WAL sender waits for transactions during commit:
The results indicate that the change did not deliver the expected
performance improvements. On the contrary:
- The subscriber's TPS showed no improvement, there was a performance
reduction of ~58%, consistent with previous observations shared at
[1]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
- Additionally, the publisher's TPS was impacted, showing a
performance drop of 7-8%.
Below are the test details:
Test setup:
- Applied v17 patch-set atop pgHead and then the top-up patch to wait
on publisher side. The top-up patch is attached.
- Created a Pub-Sub setup with configurations -
autovacuum = false
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
track_commit_timestamp = on (only on sub node)
- To avoid the update/delete_missing conflicts, distinct initial
pgbench tables were created on the publisher and subscriber nodes. On
the publisher side, the tables were renamed: pgbench_XXX ->
pgbench_pub_XXX. Initial data was inserted with scale=100 on both
nodes.
(The scripts used to run the tests - "measure.sh" and "setup.sh" are attached)
Test Run:
- Ran pgbench on both pub and sub simultaneously, on different tables.
Observations:
- No improvement observed in sub's TPS performance, as TPS reduced by -58%.
- The publisher's TPS was also reduced by -8%.
Results:
Run# | Pub's TPS | Sub's TPS
1 | 30279.48756 | 13397.7329
2 | 29634.12687 | 13316.93142
3 | 29350.39389 | 13264.91436
4 | 29700.10173 | 13394.91969
5 | 30121.19058 | 13380.86673
Median | 29700.10173 | 13380.86673
Regression | -8% | -58%
Perf analysis for Sub and walsender:
a) The sub-side backend process shows high time spent in heap tuple
scan, confirming the cause is dead_tuple accumulation.
```
....
--68.69%--ExecModifyTable
--60.92%--ExecScan
--60.75%--IndexNext
--60.55%--index_getnext_slot
--58.12%--index_fetch_heap
--57.83%--heapam_index_fetch_tuple
--40.11%--heap_hot_search_buffer
....
```
b) Collected walsender profile and it shows overall 2-3% higher time
spent in wait. Attached the walsender profiles with and without top-up
patch.
~~~~
[1]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
--
Thanks,
Nisha
Attachments:
0001-wait-for-committing-txns-to-finish-in-walsender.patchapplication/x-patch; name=0001-wait-for-committing-txns-to-finish-in-walsender.patchDownload
From f903160de5ce67e514dd063a30ffb27faf5dcc3a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 16 Dec 2024 11:32:25 +0800
Subject: [PATCH] wait for committing txns to finish in walsender
---
src/backend/replication/logical/worker.c | 47 ++++++++++--------------
src/backend/replication/walsender.c | 27 +++++++++++---
src/backend/storage/ipc/procarray.c | 16 ++------
3 files changed, 44 insertions(+), 46 deletions(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6406be4f17..4abf3ded95 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3771,8 +3771,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* to avoid accumulating dead rows when the worker is
* busy.
*/
- if (can_advance_nonremovable_xid(&data))
- maybe_advance_nonremovable_xid(&data, false);
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3796,9 +3795,6 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
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);
/*
@@ -4291,28 +4287,7 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
if (!status_received)
return;
- if (!FullTransactionIdIsValid(data->last_phase_at))
- data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
- data->remote_nextxid);
-
- /* Compute the epoch of the remote oldest running transaction ID */
- if (data->remote_oldestxid > data->remote_nextxid)
- remote_epoch--;
-
- remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
- 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;
+ data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
/* process the next phase */
maybe_advance_nonremovable_xid(data, false);
@@ -4327,6 +4302,24 @@ wait_for_local_flush(RetainConflictInfoData *data)
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
+ /*
+ * Update the remote flush position if we are applying changes in a loop
+ * and. Otherwise, the position is only updated when the feedback
+ * is sent to the server.
+ */
+ if (data->last_recv_time)
+ {
+ 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;
+ }
+
/*
* We expect the publisher and subscriber clocks to be in sync using time
* sync service like NTP. Otherwise, we will advance this worker's
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 8c18264f58..eafd658728 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -2676,7 +2676,7 @@ ProcessStandbyPSRequestMessage(void)
{
XLogRecPtr lsn = InvalidXLogRecPtr;
TransactionId oldestXidInCommit;
- FullTransactionId nextFullXid;
+ TransactionId nextXid = InvalidTransactionId;
WalSnd *walsnd = MyWalSnd;
TimestampTz replyTime;
@@ -2693,8 +2693,26 @@ ProcessStandbyPSRequestMessage(void)
walsnd->replyTime = replyTime;
SpinLockRelease(&walsnd->mutex);
- oldestXidInCommit = GetOldestTransactionIdInCommit();
- nextFullXid = ReadNextFullTransactionId();
+ for (;;)
+ {
+ oldestXidInCommit = GetOldestTransactionIdInCommit();
+
+ if (!TransactionIdIsValid(oldestXidInCommit))
+ break;
+
+ if (!TransactionIdIsValid(nextXid))
+ nextXid = XidFromFullTransactionId(ReadNextFullTransactionId());
+ else if (TransactionIdPrecedesOrEquals(nextXid, oldestXidInCommit))
+ break;
+
+ (void) WaitLatch(MyLatch,
+ WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
+ 10L,
+ WAIT_EVENT_LOGICAL_PARALLEL_APPLY_STATE_CHANGE);
+
+ CHECK_FOR_INTERRUPTS();
+ }
+
lsn = GetXLogWriteRecPtr();
elog(DEBUG2, "sending primary status");
@@ -2703,9 +2721,6 @@ ProcessStandbyPSRequestMessage(void)
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 */
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index ae6da9fd91..3bb7eaa93c 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2937,22 +2937,11 @@ GetOldestTransactionIdInCommit(void)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
- TransactionId oldestXidInCommit;
+ TransactionId oldestXidInCommit = InvalidTransactionId;
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.
*/
@@ -2970,7 +2959,8 @@ GetOldestTransactionIdInCommit(void)
continue;
if ((proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) != 0 &&
- TransactionIdPrecedes(xid, oldestXidInCommit))
+ (!TransactionIdIsValid(oldestXidInCommit) ||
+ TransactionIdPrecedes(xid, oldestXidInCommit)))
oldestXidInCommit = xid;
/*
--
2.30.0.windows.2
On Friday, January 3, 2025 2:36 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
I have one comment on the 0001 patch:
Thanks for the comments!
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples from being + * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transaction ID + * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID wraparound
happens. But once we set the oldest-nonremovable-xid it prevents XIDs from
being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.
I think the issue is that the launcher may create the replication slot after
the apply worker has already set the 'oldest_nonremovable_xid' because the
launcher are doing that asynchronously. So, Before the slot is created, there's
a window where transaction IDs might wrap around. If initially the apply worker
has computed a candidate_xid (755) and the xid wraparound before the launcher
creates the slot, causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher could
advance the xmin to 755 which cause the dead tuples to be removed prematurely.
(We are trying to reproduce this to ensure that it's a real issue and will
share after finishing)
We thought of another approach, which is to create/drop this slot first as
soon as one enables/disables detect_update_deleted (E.g. create/drop slot
during DDL). But it seems complicate to control the concurrent slot
create/drop. For example, if one backend A enables detect_update_deteled, it
will create a slot. But if another backend B is disabling the
detect_update_deteled at the same time, then the newly created slot may be
dropped by backend B. I thought about checking the number of subscriptions that
enables detect_update_deteled before dropping the slot in backend B, but the
subscription changes caused by backend A may not visable yet (e.g. not
committed yet).
Does that make sense to you, or do you have some other ideas?
Best Regards,
Hou zj
On Mon, Jan 6, 2025 at 4:52 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, January 3, 2025 2:36 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
I have one comment on the 0001 patch:
Thanks for the comments!
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples from being + * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transaction ID + * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID wraparound
happens. But once we set the oldest-nonremovable-xid it prevents XIDs from
being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot after
the apply worker has already set the 'oldest_nonremovable_xid' because the
launcher are doing that asynchronously. So, Before the slot is created, there's
a window where transaction IDs might wrap around. If initially the apply worker
has computed a candidate_xid (755) and the xid wraparound before the launcher
creates the slot, causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher could
advance the xmin to 755 which cause the dead tuples to be removed prematurely.
(We are trying to reproduce this to ensure that it's a real issue and will
share after finishing)
I tried to reproduce the issue described above, where an
xid_wraparound occurs before the launcher creates the conflict slot,
and the apply worker retains a very old xid (from before the
wraparound) as its oldest_nonremovable_xid.
In this scenario, the launcher will not update the apply worker's
older epoch xid (oldest_nonremovable_xid = 755) as the conflict slot's
xmin. This is because advance_conflict_slot_xmin() ensures proper
handling by comparing the full 64-bit xids. However, this could lead
to real issues if 32-bit TransactionID were used instead of 64-bit
FullTransactionID. The detailed test steps and results are as below:
Setup: A Publisher-Subscriber setup with logical replication.
Steps done to reproduce the test scenario -
On Sub -
1) Created a subscription with detect_update_deleted=off, so no
conflict slot to start with.
2) Attached gdb to the launcher and put a breakpoint at
advance_conflict_slot_xmin().
3) Run "alter subscription ..... (detect_update_deleted=ON);"
4) Stopped the launcher at the start of the
"advance_conflict_slot_xmin()", and blocked the creation of the
conflict slot.
5) Attached another gdb session to the apply worker and made sure it
has set an oldest_nonremovable_xid . In
"maybe_advance_nonremovable_xid()" -
(gdb) p MyLogicalRepWorker->oldest_nonremovable_xid
$3 = {value = 760}
-- so apply worker's oldest_nonremovable_xid = 760
6) Consumed ~4.2 billion xids to let the xid_wraparound happen. After
the wraparound, the next_xid was "705", which is less than "760".
7) Released the launcher from gdb, but the apply_worker still stopped in gdb.
8) The slot gets created with xmin=705 :
postgres=# select slot_name, slot_type, active, xmin, catalog_xmin,
restart_lsn, inactive_since, confirmed_flush_lsn from
pg_replication_slots;
slot_name | slot_type | active | xmin | catalog_xmin |
restart_lsn | inactive_since | confirmed_flush_lsn
-----------------------+-----------+--------+------+--------------+-------------+----------------+---------------------
pg_conflict_detection | physical | t | 705 | |
| |
(1 row)
Next, when launcher tries to advance the slot's xmin in
advance_conflict_slot_xmin() with new_xmin as the apply worker's
oldest_nonremovable_xid(760), it returns without updating the slot's
xmin because of below check -
````
if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return false;
````
we are comparing the full xids (64-bit) in
FullTransactionIdPrecedesOrEquals() and in this case the values are:
new_xmin=760
full_xmin=4294968001 (w.r.t. xid=705)
As "760 <= 4294968001", the launcher will return from here and not
update the slot's xmin to "760". Above check will always be true in
such scenarios.
Note: The launcher would have updated the slot's xmin to 760 if 32-bit
XIDs were being compared, i.e., "760 <= 705".
--
Thanks,
Nisha
On Mon, Jan 6, 2025 at 3:22 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, January 3, 2025 2:36 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
I have one comment on the 0001 patch:
Thanks for the comments!
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples from being + * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transaction ID + * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID wraparound
happens. But once we set the oldest-nonremovable-xid it prevents XIDs from
being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot after
the apply worker has already set the 'oldest_nonremovable_xid' because the
launcher are doing that asynchronously. So, Before the slot is created, there's
a window where transaction IDs might wrap around. If initially the apply worker
has computed a candidate_xid (755) and the xid wraparound before the launcher
creates the slot, causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher could
advance the xmin to 755 which cause the dead tuples to be removed prematurely.
(We are trying to reproduce this to ensure that it's a real issue and will
share after finishing)
The slot's first xmin is calculated by
GetOldestSafeDecodingTransactionId(false). The initial computed
cancidate_xid could be newer than this xid?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tuesday, January 7, 2025 2:00 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Mon, Jan 6, 2025 at 3:22 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Friday, January 3, 2025 2:36 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I have one comment on the 0001 patch:
Thanks for the comments!
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts + when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples frombeing
+ * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to + advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transactionID
+ * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID
wraparound happens. But once we set the oldest-nonremovable-xid it
prevents XIDs from being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot
after the apply worker has already set the 'oldest_nonremovable_xid'
because the launcher are doing that asynchronously. So, Before the
slot is created, there's a window where transaction IDs might wrap
around. If initially the apply worker has computed a candidate_xid
(755) and the xid wraparound before the launcher creates the slot,
causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher
could advance the xmin to 755 which cause the dead tuples to be removedprematurely.
(We are trying to reproduce this to ensure that it's a real issue and
will share after finishing)The slot's first xmin is calculated by
GetOldestSafeDecodingTransactionId(false). The initial computed
cancidate_xid could be newer than this xid?
I think the issue occurs when the slot is created after an XID wraparound. As a
result, GetOldestSafeDecodingTransactionId() returns the current XID
(after wraparound), which appears older than the computed candidate_xid (e.g.,
oldest_nonremovable_xid). Nisha has shared detailed steps to reproduce the
issue in [1]/messages/by-id/CABdArM6P0zoEVRN+3YHNET_oOaAVOKc-EPUnXiHkcBJ-uDKQVw@mail.gmail.com. What do you think ?
[1]: /messages/by-id/CABdArM6P0zoEVRN+3YHNET_oOaAVOKc-EPUnXiHkcBJ-uDKQVw@mail.gmail.com
Best Regards,
Hou zj
On Mon, Jan 6, 2025 at 10:40 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, January 7, 2025 2:00 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Mon, Jan 6, 2025 at 3:22 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Friday, January 3, 2025 2:36 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I have one comment on the 0001 patch:
Thanks for the comments!
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts + when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples frombeing
+ * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to + advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transactionID
+ * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID
wraparound happens. But once we set the oldest-nonremovable-xid it
prevents XIDs from being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot
after the apply worker has already set the 'oldest_nonremovable_xid'
because the launcher are doing that asynchronously. So, Before the
slot is created, there's a window where transaction IDs might wrap
around. If initially the apply worker has computed a candidate_xid
(755) and the xid wraparound before the launcher creates the slot,
causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher
could advance the xmin to 755 which cause the dead tuples to be removedprematurely.
(We are trying to reproduce this to ensure that it's a real issue and
will share after finishing)The slot's first xmin is calculated by
GetOldestSafeDecodingTransactionId(false). The initial computed
cancidate_xid could be newer than this xid?I think the issue occurs when the slot is created after an XID wraparound. As a
result, GetOldestSafeDecodingTransactionId() returns the current XID
(after wraparound), which appears older than the computed candidate_xid (e.g.,
oldest_nonremovable_xid). Nisha has shared detailed steps to reproduce the
issue in [1]. What do you think ?
I agree that the scenario Nisha shared could happen with the current
patch. On the other hand, I think that if slot's initial xmin is
always newer than or equal to the initial computed non-removable-xid
(i.e., the oldest of workers' oldest_nonremovable_xid values), we can
always use slot's first xmin. And I think it might be true while I'm
concerned the fact that worker's oldest_nonremoable_xid and the slot's
initial xmin is calculated differently (GetOldestActiveTransactionId()
and GetOldestSafeDecodingTransactionId(), respectively). That way,
subsequent comparisons between slot's xmin and computed candidate_xid
won't need to take care of the epoch. IOW, the worker's
non-removable-xid values effectively are not used until the slot is
created.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Fri, Jan 3, 2025 at 11:22 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
5. + + <varlistentry id="sql-createsubscription-params-with-detect-update-deleted"> + <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term> + <listitem> + <para> + Specifies whether the detection of <xref linkend="conflict-update-deleted"/> + is enabled. The default is <literal>false</literal>. If set to + true, the dead tuples on the subscriber that are still useful for + detecting <xref linkend="conflict-update-deleted"/> + are retained,One of the purposes of retaining dead tuples is to detect
update_delete conflict. But, I also see the following in 0001's commit
message: "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."
which indicates there are other cases where retaining dead tuples can
help. So, I was thinking about whether to name this new option as
retain_dead_tuples or something along those lines?
The other possible option name could be retain_conflict_info.
Sawada-San, and others, do you have any preference for the name of
this option?
--
With Regards,
Amit Kapila.
On Tuesday, January 7, 2025 3:05 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Mon, Jan 6, 2025 at 10:40 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, January 7, 2025 2:00 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
Hi,
On Mon, Jan 6, 2025 at 3:22 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:
On Friday, January 3, 2025 2:36 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
I have one comment on the 0001 patch:
Thanks for the comments!
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts + when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples frombeing
+ * prematurely frozen or removed. The logical replication
launcher
+ * asynchronously collects this info to determine whether to + advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID.This is
+ * critical because without considering the epoch, the
transaction
ID
+ * alone may appear as if it is in the future due to transaction
ID
+ * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID
wraparound happens. But once we set the oldest-nonremovable-xid it
prevents XIDs from being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot
after the apply worker has already set the 'oldest_nonremovable_xid'
because the launcher are doing that asynchronously. So, Before the
slot is created, there's a window where transaction IDs might wrap
around. If initially the apply worker has computed a candidate_xid
(755) and the xid wraparound before the launcher creates the slot,
causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher
could advance the xmin to 755 which cause the dead tuples to beremoved
prematurely.
(We are trying to reproduce this to ensure that it's a real issue and
will share after finishing)The slot's first xmin is calculated by
GetOldestSafeDecodingTransactionId(false). The initial computed
cancidate_xid could be newer than this xid?I think the issue occurs when the slot is created after an XID wraparound. As
a
result, GetOldestSafeDecodingTransactionId() returns the current XID
(after wraparound), which appears older than the computed candidate_xid(e.g.,
oldest_nonremovable_xid). Nisha has shared detailed steps to reproduce the
issue in [1]. What do you think ?I agree that the scenario Nisha shared could happen with the current
patch. On the other hand, I think that if slot's initial xmin is
always newer than or equal to the initial computed non-removable-xid
(i.e., the oldest of workers' oldest_nonremovable_xid values), we can
always use slot's first xmin. And I think it might be true while I'm
concerned the fact that worker's oldest_nonremoable_xid and the slot's
initial xmin is calculated differently (GetOldestActiveTransactionId()
and GetOldestSafeDecodingTransactionId(), respectively). That way,
subsequent comparisons between slot's xmin and computed candidate_xid
won't need to take care of the epoch. IOW, the worker's
non-removable-xid values effectively are not used until the slot is
created.
I might be missing something, so could you please elaborate a bit more on this
idea?
Initially, I thought you meant delaying the initialization of slot.xmin until
after the worker computes the oldest_nonremovable_xid. However, I think the
same issue would occur with this approach as well [1], with the difference
being that the slot would directly use a future XID as xmin, which seems
inappropriate to me.
Or do you mean opposite that we delay the initialization of
oldest_nonremovable_xid after the creation of the slot ?
[1]:
So, Before the slot is created, there's a window where transaction IDs might
wrap around. If initially the apply worker has computed a candidate_xid (755)
and the xid wraparound before the launcher creates the slot, causing the new
current xid to be (740), then the old candidate_xid(755) looks like a xid in
the future, and the launcher could advance the xmin to 755 which cause the
dead tuples to be removed prematurely.
Best Regards,
Hou zj
On Mon, Jan 6, 2025 at 4:52 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, January 3, 2025 2:36 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples from being + * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transaction ID + * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID wraparound
happens. But once we set the oldest-nonremovable-xid it prevents XIDs from
being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot after
the apply worker has already set the 'oldest_nonremovable_xid' because the
launcher are doing that asynchronously. So, Before the slot is created, there's
a window where transaction IDs might wrap around. If initially the apply worker
has computed a candidate_xid (755) and the xid wraparound before the launcher
creates the slot, causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher could
advance the xmin to 755 which cause the dead tuples to be removed prematurely.
(We are trying to reproduce this to ensure that it's a real issue and will
share after finishing)We thought of another approach, which is to create/drop this slot first as
soon as one enables/disables detect_update_deleted (E.g. create/drop slot
during DDL). But it seems complicate to control the concurrent slot
create/drop. For example, if one backend A enables detect_update_deteled, it
will create a slot. But if another backend B is disabling the
detect_update_deteled at the same time, then the newly created slot may be
dropped by backend B. I thought about checking the number of subscriptions that
enables detect_update_deteled before dropping the slot in backend B, but the
subscription changes caused by backend A may not visable yet (e.g. not
committed yet).
This means that for the transaction whose changes are not yet visible,
we may have already created the slot and the backend B would end up
dropping it. Is it possible that during the change of this new option
via DDL, we take AccessExclusiveLock on pg_subscription as we do in
DropSubscription() to ensure that concurrent transactions can't drop
the slot? Will that help in solving the above scenario?
The second idea could be that each worker first checks whether a slot
exists along with a subscription flag (new option). Checking the
existence of a slot each time would be costly, so we somehow need to
cache it. But if we do that then we need to invent some cache
invalidation mechanism for the slot. I am not sure if we can design a
race-free mechanism for that. I mean we need to think of a solution
for race conditions between the launcher and apply workers to ensure
that after dropping the slot, launcher doesn't recreate the slot (say
if some subscription enables this option) before all the workers can
clear their existing values of oldest_nonremovable_xid.
The third idea to avoid the race condition could be that in the
function InitializeLogRepWorker() after CommitTransactionCommand(), we
check if the retain_dead_tuples flag is true for MySubscription then
we check whether the system slot exists. If exits then go ahead,
otherwise, wait till the slot is created. It could be some additional
cycles during worker start up but it is a one-time effort and that too
only when the flag is set. In addition to this, we anyway need to
create the slot in the launcher before launching the workers, and
after re-reading the subscription, the change in retain_dead_tuples
flag (off->on) should cause the worker restart.
Now, in the third idea, the issue can still arise if, after waiting
for the slot to be created, the user sets the retain_dead_tuples to
false and back to true again immediately. Because the launcher may
have noticed the "retain_dead_tuples=false" operation and dropped the
slot, while the apply worker has not noticed and still holds an old
candidate_xid. The xid may wraparound in this window before setting
the retain_dead_tuples back to true. And, the apply worker would not
restart because after it calls maybe_reread_subscription(), the
retain_dead_tuples would have been set back to true again. Again, to
avoid this race condition, the launcher can wait for each worker to
reset the oldest_nonremovamble_xid before dropping the slot.
Even after doing the above, the third idea could still have another
race condition:
1. The launcher creates the replication slot and starts a worker with
retain_dead_tuples = true, the worker is waiting for publish status
and has not set oldest_nonremovable_xid.
2. The user set the option retain_dead_tuples to false, the launcher
noticed that and drop the replication slot.
3. The worker received the status and set oldest_nonremovable_xid to a
valid value (say 750).
4. Xid wraparound happened at this point and say new_available_xid becomes 740
5. User set retain_dead_tuples = true again.
After the above steps, the apply worker holds an old
oldest_nonremovable_xid (750) and will not restart if it does not call
maybe_reread_subscription() before step 5. So, such a case can again
create a problem of incorrect slot->xmin value. We can probably try to
find some way to avoid this race condition as well but I haven't
thought more about this as this idea sounds a bit risky and bug-prone
to me.
Among the above ideas, the first idea of taking AccessExclusiveLock on
pg_subscription sounds safest to me. I haven't evaluated the changes
for the first approach so I could be missing something that makes it
difficult to achieve but I think it is worth investigating unless we
have better ideas or we think that the current approach used in patch
to use FullTransactionId is okay.
--
With Regards,
Amit Kapila.
On Wed, 25 Dec 2024 at 08:13, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, December 23, 2024 2:15 PM Kuroda, Hayato/黒田 隼人 <kuroda.hayato@fujitsu.com> wrote:
Dear Hou,
Thanks for updating the patch. Few comments:
Thanks for the comments!
02. ErrorOnReservedSlotName()
Currently the function is callsed from three points -
create_physical_replication_slot(),
create_logical_replication_slot() and CreateReplicationSlot().
Can we move them to the ReplicationSlotCreate(), or combine into
ReplicationSlotValidateName()?I am not sure because moving the check into these functions because that would
prevent the launcher from creating the slot as well unless we add a new
parameter for these functions, but I am not sure if it's worth it at this
stage.03. advance_conflict_slot_xmin()
```
Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
```Assuming the case that the launcher crashed just after
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT).
After the restart, the slot can be acquired since
SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT)
is true, but the process would fail the assert because data.xmin is still invalid.I think we should re-create the slot when the xmin is invalid. Thought?
After thinking more, the standard approach to me would be to mark the slot as
EPHEMERAL during creation and persist it after initializing, so changed like
that.05. check_remote_recovery()
Can we add a test case related with this?
I think the code path is already tested, and I am a bit unsure if we want to setup
a standby to test the ERROR case, so didn't add this.---
Attach the new version patch set which addressed all other comments.
I was doing backward compatibility test by creating publication in
PG17 and subscription with the patch on HEAD:
Currently, we are able to create subscription with
detect_update_deleted option for a publication on PG17:
postgres=# create subscription sub1 connection 'dbname=postgres
host=localhost port=5432' publication pub1 with
(detect_update_deleted=true);
NOTICE: created replication slot "sub1" on publisher
CREATE SUBSCRIPTION
This should not be allowed now as the subscriber will now request
publisher status from the publisher for which handling is not
available in the publisher:
+static void
+request_publisher_status(RetainConflictInfoData *data)
+{
...
+ pq_sendbyte(request_message, 'p');
+ pq_sendint64(request_message, GetCurrentTimestamp());
...
+}
I felt this should not be allowed.
Regards,
Vignesh
Dear Amit, Hou,
BTW, it is not clear how retaining dead tuples will help the detection
update_origin_differs. Will it happen when the tuple is inserted or
updated on the subscriber and then when we try to update the same
tuple due to remote update, the commit_ts information of the xact is
not available because the same is already removed by vacuum? This
should happen for the update case for the new row generated by the
update operation as that will be used in comparison. Can you please
show it be a test case even if it is manual?
I've confirmed that retaining dead tuples is helpful for the origin_differ detection.
I considered a workload to prove it.
## Workload
I ran below steps:
1. Setup a publisher. There was a table and it had a tuple.
2. Setup a subscriber. There was a table and the same tuple was inserted by itself (not replicated).
The subscription option was "copy_data = off", and GUC setting was "track_commit_timestamp=on".
3. Installed "xid_wraparound" extension on the subscriber.
4. Called `SELECT consume_xids(60000000);` on the subscriber side to advance xid.
5. Ran VACUUM FREEZE for all the databases on the subscriber side.
6. Updated the tuple on the publisher side.
The key idea is that commit_ts entries could be removed when the given
transaction ID is frozen. In this workload, 60 millon transaction IDs are consumed
and this is bit larger than vacuum_freeze_min_age.
I.e., step 5 can freeze a tuple which was inserted to the subscriber at step 2.
Then, we run VACUUM FREEZE command for databases to advance pg_database.datfrozenxid.
This command can teach commit_ts module that entries for old transactions can be truncated.
Attached script automate the test.
## Result
When detect_update_deleted of the subscription was set to false,
update_origin_differ was not detected on the subscriber.
In contrast, when detect_update_deleted was true, it was detectable:
```
LOG: conflict detected on relation "public.foo": conflict=update_origin_differs
DETAIL: Updating the row that was modified locally in transaction 745 at ...
Existing local tuple (1, 1); remote tuple (1, 2); replica identity (a)=(1).
```
Based on the fact, we can conclude that the option is also helpful for detecting
origin_differ conflicts.
Can't it happen for delete_origin_differs as well for the same reason?
Right. I've also tested with almost the same way as above, and I got the same result.
You can confirm by modifying the last statement in the attached script.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Attachments:
On Friday, January 3, 2025 1:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 25, 2024 at 8:13 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the new version patch set which addressed all other comments.
Some more miscellaneous comments:
Thanks for the comments!
============================= 1. @@ -1431,9 +1431,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. @@ -1536,7 +1536,7 @@ RecordTransactionCommit(void) */ if (markXidCommitted) { - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION();The comments related to this change should be updated in EndPrepare()
and RecordTransactionCommitPrepared(). They still refer to the
DELAY_CHKPT_START flag. We should update the comments explaining why
a
similar change is not required for prepare or commit_prepare, if there
is one.
After considering more, I think we need to use the new flag in
RecordTransactionCommitPrepared() as well, because it is assigned a commit
timestamp and would be replicated as normal transaction if sub's two_phase is
not enabled.
3. +FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot, + TransactionId *delete_xid, + RepOriginId *delete_origin, + TimestampTz *delete_time) ... ... + /* 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); + + if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD) + dead = true; + + LockBuffer(buf, BUFFER_LOCK_UNLOCK); + + if (!dead) + continue;Why do we need to check only for HEAPTUPLE_RECENTLY_DEAD and not
HEAPTUPLE_DEAD? IIUC, we came here because we couldn't find the live
tuple, now whether the tuple is DEAD or RECENTLY_DEAD, why should it
matter to detect update_delete conflict?
The HEAPTUPLE_DEAD could indicate tuples whose inserting transaction was
aborted, in which case we could not get the commit timestamp or origin for the
transaction. Or it could indicate tuples deleted by a transaction older than
oldestXmin(we would take the new replication slot's xmin into account when
computing this value), which means any subsequent transaction would have commit
timestamp later than that old delete transaction, so I think it's OK to ignore
this dead tuple and even detect update_missing because the resolution is to
apply the subsequent UPDATEs anyway (assuming we are using last update win
strategy). I added some comments along these lines in the patch.
5. + + <varlistentry id="sql-createsubscription-params-with-detect-update-deleted"> + <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term> + <listitem> + <para> + Specifies whether the detection of <xref linkend="conflict-update-deleted"/> + is enabled. The default is <literal>false</literal>. If set to + true, the dead tuples on the subscriber that are still useful for + detecting <xref linkend="conflict-update-deleted"/> + are retained,One of the purposes of retaining dead tuples is to detect
update_delete conflict. But, I also see the following in 0001's commit
message: "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."
which indicates there are other cases where retaining dead tuples can
help. So, I was thinking about whether to name this new option as
retain_dead_tuples or something along those lines?
I used the retain_conflict_info in this version as it looks more general and we
are already using similar name in patch(RetainConflictInfoData), but we can
change it later if people have better ideas.
Attached the V19 patch which addressed comments in [1]/messages/by-id/CAA4eK1JpJPQAPcnrVBNJ8CMaFfO9A9P6-DmXbwSfMAVrjMi5Qw@mail.gmail.com[2]/messages/by-id/CALDaNm1_V+WPThOkZy+R9_sWgHH5H6hN6UtEmq4Mj3QbUc3G8g@mail.gmail.com[3]/messages/by-id/CAA4eK1JkYpDUCFgGNEg=O4m6XjQ-9oohxrohgpockKuy7eo9gA@mail.gmail.com[4]/messages/by-id/CALDaNm3whh00AN58Azsps6+NLsYgqL6w2hz6wmcqSw5uiYqAbA@mail.gmail.com[5]/messages/by-id/CALDaNm08M6CRZkK=BtVfS1=+zV2Qayg+fnYXQEPBiMOQ39m6sg@mail.gmail.com[6]/messages/by-id/CAA4eK1+XfGqYRJPFk0wUHAh3mkJ59Tj2q+gXctyk7SKiASHgFA@mail.gmail.com[7]/messages/by-id/CAA4eK1+dAJNPJWd_+OR7s+4rzSs48Jaoa2+0WNe+=9VQrCh4_A@mail.gmail.com.
The point to remove FullTransactionID have not been addressed as the discussion is
still on going.
[1]: /messages/by-id/CAA4eK1JpJPQAPcnrVBNJ8CMaFfO9A9P6-DmXbwSfMAVrjMi5Qw@mail.gmail.com
[2]: /messages/by-id/CALDaNm1_V+WPThOkZy+R9_sWgHH5H6hN6UtEmq4Mj3QbUc3G8g@mail.gmail.com
[3]: /messages/by-id/CAA4eK1JkYpDUCFgGNEg=O4m6XjQ-9oohxrohgpockKuy7eo9gA@mail.gmail.com
[4]: /messages/by-id/CALDaNm3whh00AN58Azsps6+NLsYgqL6w2hz6wmcqSw5uiYqAbA@mail.gmail.com
[5]: /messages/by-id/CALDaNm08M6CRZkK=BtVfS1=+zV2Qayg+fnYXQEPBiMOQ39m6sg@mail.gmail.com
[6]: /messages/by-id/CAA4eK1+XfGqYRJPFk0wUHAh3mkJ59Tj2q+gXctyk7SKiASHgFA@mail.gmail.com
[7]: /messages/by-id/CAA4eK1+dAJNPJWd_+OR7s+4rzSs48Jaoa2+0WNe+=9VQrCh4_A@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v19-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v19-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 2f81969107fd4e1cc1c7da81cb12b2d4e78266b6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v19 1/3] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 19 +-
src/backend/access/transam/xact.c | 6 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 482 ++++++++++++++++++++-
src/backend/replication/walsender.c | 54 +++
src/backend/storage/ipc/procarray.c | 59 +++
src/include/replication/worker_internal.h | 16 +
src/include/storage/proc.h | 5 +
src/include/storage/procarray.h | 1 +
src/tools/pgindent/typedefs.list | 2 +
11 files changed, 726 insertions(+), 9 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..49a4d3ab1b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1..386bfde916 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1191,6 +1191,13 @@ EndPrepare(GlobalTransaction gxact)
START_CRIT_SECTION();
+ /*
+ * It is OK to use the DELAY_CHKPT_START flag instead of
+ * DELAY_CHKPT_IN_COMMIT. Currently, the DELAY_CHKPT_IN_COMMIT flag is
+ * used for conflict detection and resolution in logical replication,
+ * relying on the commit timestamp. Therefore, prepared transactions that
+ * lack a commit timestamp are not taken into account.
+ */
Assert((MyProc->delayChkptFlags & DELAY_CHKPT_START) == 0);
MyProc->delayChkptFlags |= DELAY_CHKPT_START;
@@ -2356,7 +2363,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2383,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2396,10 @@ 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;
+
+ committs = GetCurrentTimestamp();
/*
* Emit the XLOG commit record. Note that we mark 2PC commits as
@@ -2439,7 +2448,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 d331ab90d7..e3753d5d21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a..31ebef831c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7af..a93c473238 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,60 @@ 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 */
+
+ /*
+ * 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 +394,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 +441,17 @@ 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 inline bool can_advance_nonremovable_xid(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,
@@ -3573,6 +3646,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
@@ -3651,6 +3725,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);
@@ -3677,6 +3753,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3776,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);
@@ -3706,6 +3814,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)
{
/*
@@ -3740,6 +3853,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,
@@ -3803,6 +3923,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
@@ -3838,7 +3960,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;
@@ -3916,6 +4037,365 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * 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;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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;
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
+/*
+ * 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 a0782b1bbf..1299ed1eb5 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f88..064841b482 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 30b2775952..97c24eb42b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,22 @@ 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.
+ *
+ * Note that this info cannot directly protect dead tuples from being
+ * prematurely frozen or removed. The logical replication launcher
+ * asynchronously collects this info to determine whether to advance the
+ * xmin value of its replication slot.
+ *
+ * We need to use FullTransactionId here because without considering the
+ * epoch, the transaction ID alone may appear as if it is in the future
+ * due to the transaction ID wraparound.
+ */
+ 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 20777f7d5a..9212f2531b 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,14 @@ 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.
*/
#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 ef0b733ebe..bd8b17a6d0 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 e1c4f913f8..b5ea6bb5b8 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2474,6 +2474,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v19-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v19-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From e434ba938e49b708a75d030a159cd1ca1b8ad41a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v19 2/3] 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 | 216 ++++++++++++++++++
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/slot.c | 34 ++-
src/include/replication/slot.h | 11 +-
10 files changed, 279 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 740ff5d504..71f0f97420 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4877,6 +4877,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581a..feb026d0c5 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b..e20b48a49a 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..01b37a5a3e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for the
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea20..d49ba78b13 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e..3cf1e05539 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 31ebef831c..c607bf7988 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1119,6 +1128,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1159,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();
@@ -1167,14 +1181,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1250,35 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated;
+
+ if (!can_advance_xmin)
+ xmin = InvalidFullTransactionId;
+
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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,150 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ /*
+ * Acquire the slot if it hasn't been acquired yet. If the slot is not yet
+ * created, create it.
+ */
+ if (!MyReplicationSlot)
+ {
+ if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ {
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true);
+ }
+ else
+ {
+ TransactionId xmin_horizon;
+
+ /*
+ * Initially create persistent slot as ephemeral - that allows us
+ * to nicely handle errors during initialization because it'll get
+ * dropped if this transaction fails. We'll make it persistent
+ * after safely initializing the xmin of the slot.
+ */
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+ }
+ }
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
+
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf..64b43346a9 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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/slot.c b/src/backend/replication/slot.c
index b30e0473e1..3a3884b382 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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/slot.h b/src/include/replication/slot.h
index bf62b36ad0..ea0485efa2 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.
*
@@ -261,7 +268,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);
--
2.30.0.windows.2
v19-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v19-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From a9805d3f67e146341c3ba0d30a1f70fbc1793249 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:42:39 +0800
Subject: [PATCH v19 3/5] Add a retain_conflict_info option to subscriptions
This patch adds a subscription option allowing users to specify whether dead
tuples on the subscriber, which are 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
dead tuples 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_dead_tuples
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 | 26 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 138 ++++++++++++++--
src/backend/replication/logical/launcher.c | 26 ++-
src/backend/replication/logical/worker.c | 4 +
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 37 ++++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 5 +-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 14 +-
src/include/catalog/pg_subscription.h | 5 +
src/test/regress/expected/subscription.out | 181 ++++++++++++---------
src/test/regress/sql/subscription.sql | 19 +++
19 files changed, 415 insertions(+), 120 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index bf3cee08a9..2910787407 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8052,6 +8052,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..7de99a3651 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2160,7 +2160,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..a810137a8c 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 01b37a5a3e..35ad70d6a2 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,32 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 89bf5ec933..614f9b42ca 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 da9a8fe99f..97f9a71ffa 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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 0d37557841..3be1835b00 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_remote_recovery(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(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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_remote_recovery(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 query_remote_recovery = 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(ERROR,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("detecting update_deleted conflicts requires \"%s\" to be enabled",
+ "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();
+
+ query_remote_recovery = true;
+ }
+
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 recovery status may not be checked. Ensure
+ * this check is conducted now.
+ */
+ query_remote_recovery = 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 is not in recovery if
+ * retain_conflict_info is enabled.
+ */
+ query_remote_recovery = sub->retainconflictinfo;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1634,14 @@ 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 the recovery status of the remote server.
*
* 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 || query_remote_recovery)
{
bool must_use_password;
char *err;
@@ -1584,10 +1650,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 +1666,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_remote_recovery(wrconn, query_remote_recovery);
+
+ 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();
{
@@ -2193,6 +2266,45 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Check if the retain_conflict_info setting is enabled while the publisher is
+ * in recovery (operating as a standby server) and report an ERROR in such cases.
+ *
+ * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
+ * reasons.
+ */
+static void
+check_remote_recovery(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;
+
+ 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."));
+}
+
/*
* 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 9caa078b66..e9e8f6d4a7 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -166,6 +166,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);
@@ -1169,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1189,12 +1191,20 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * Create a replication slot to retain dead tuples if any
+ * subscription requests it. Only advance xmin when all such
+ * subscriptions are enabled.
+ */
+ if (sub->retainconflictinfo)
{
- can_advance_xmin = false;
- continue;
+ retain_dead_tuples = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1213,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_dead_tuples. 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;
@@ -1264,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_dead_tuples)
{
bool updated;
@@ -1286,6 +1297,7 @@ ApplyLauncherMain(Datum main_arg)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 18b4ca5f0f..a24d545163 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4102,6 +4102,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
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index ec0cdf4ed7..7e1af404a3 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 2e55a0e3bb..972442fbef 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -672,6 +672,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 94164f0472..3e224bf0c9 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1839,13 +1839,25 @@ check_new_cluster_subscription_configuration(void)
conn = connectToServer(&new_cluster, "template1");
res = executeQueryOrDie(conn, "SELECT setting FROM pg_settings "
- "WHERE name = 'max_replication_slots';");
+ "WHERE name IN ('max_replication_slots', 'track_commit_timestamp') "
+ "ORDER BY name;");
- if (PQntuples(res) != 1)
+ if (PQntuples(res) != 2)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_require_commit_ts)
+ {
+ if (old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+
+ if (strcmp(PQgetvalue(res, 1, 0), "f") == 0)
+ pg_fatal("\"track_commit_timestamp\" must be ture on new cluster");
+ }
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1924,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 f83ded89cb..6325bd4e16 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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_require_commit_ts;
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 require_commit_ts "
+ "FROM pg_catalog.pg_subscription");
+ else
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "'f' AS require_commit_ts "
+ "FROM pg_catalog.pg_subscription");
+
+ i_nsub = PQfnumber(res, "nsub");
+ i_require_commit_ts = PQfnumber(res, "require_commit_ts");
+
+ cluster->nsubs = atoi(PQgetvalue(res, 0, i_nsub));
+ cluster->sub_require_commit_ts = (strcmp(PQgetvalue(res, 0, i_require_commit_ts), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 53f693c2d4..117155031f 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,9 @@ typedef struct
uint32 bin_version; /* version returned from pg_ctl */
const char *tablespace_suffix; /* directory specification */
int nsubs; /* number of subscriptions */
+ bool sub_require_commit_ts; /* whether the subscription requires
+ * the track_commit_timestamp setting
+ * to be enabled. */
} ClusterInfo;
@@ -430,7 +433,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/psql/describe.c b/src/bin/psql/describe.c
index 2657abdc72..9410414b82 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,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("Detect update deleted"));
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 bbd08770c3..9d07fbf07a 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,9 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> 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");
+ COMPLETE_WITH("binary", "retain_conflict_info", "disable_on_error",
+ "failover", "origin", "password_required",
+ "run_as_owner", "slot_name", "streaming",
+ "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3673,9 +3674,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
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");
+ "retain_conflict_info", "disable_on_error", "enabled",
+ "failover", "origin", "password_required",
+ "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 beaff6578a..7a4fc5ec08 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 1443e1d929..afa2f4c806 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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 | Detect update deleted | 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,47 @@ 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 | Detect update deleted | 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 | Detect update deleted | 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
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | Detect update deleted | 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)
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = true);
+ERROR: detecting update_deleted conflicts requires "track_commit_timestamp" to be enabled
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Detect update deleted | 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 007c9e7037..1088a3bd66 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,25 @@ 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);
+
+-- fail - must enable track_commit_timestamp
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = true);
+
+-- now it works
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+
+\dRs+
+
+-- fail - must enable track_commit_timestamp
+ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = true);
+
+\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;
--
2.31.1
v19-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v19-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From b16afb75ab120437a43297c4a82ccf44477ffd32 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v19 4/5] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_confl_update_deleted.pl | 156 ++++++++++++++++++
2 files changed, 157 insertions(+)
create mode 100644 src/test/subscription/t/035_confl_update_deleted.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f..d5e0e899c7 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_confl_update_deleted.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_confl_update_deleted.pl b/src/test/subscription/t/035_confl_update_deleted.pl
new file mode 100644
index 0000000000..b60092ec24
--- /dev/null
+++ b/src/test/subscription/t/035_confl_update_deleted.pl
@@ -0,0 +1,156 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# Setup a bidirectional logical replication between node_A & node_B
+###############################################################################
+
+# 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
+# replication slot's xmin value.
+my $node_A = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{track_commit_timestamp = on
+ autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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");
+
+###############################################################################
+# 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);
+
+my $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);
+
+my $last_xid = $node_A->safe_psql('postgres',
+ "SELECT xid from pg_last_committed_xact();");
+
+# Confirm that the xmin value is updated
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin = $last_xid + 1 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();
--
2.30.0.windows.2
v19-0005-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v19-0005-Support-the-conflict-detection-for-update_delete.patchDownload
From 7b597b650db82ea1b4f38d1a880e60c845bd3dba Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@fujitsu.com>
Date: Tue, 7 Jan 2025 12:10:18 +0800
Subject: [PATCH v19 5/5] 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_confl_update_deleted.pl | 16 +-
12 files changed, 260 insertions(+), 39 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 7de99a3651..76b28a514c 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d0d176cc54..878e955e62 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index cee60482b6..456e6835db 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,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 e3e4e41ac3..0009108df7 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88..0eedbd2b16 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 9d0dde45e1..5b9c2f0608 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2742,6 +2742,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2758,15 +2761,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2777,7 +2776,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2796,19 +2795,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3129,9 +3137,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3140,20 +3148,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3162,7 +3179,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3173,7 +3190,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 3245f3a8d8..80e0c8150f 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2087,7 +2087,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2109,17 +2109,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index b37e8a6f88..e6c2f1d342 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 f8a8d03e53..83ccfd1b50 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"
@@ -670,7 +671,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 37454dc951..91224b6ea6 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 3014d047fe..2f5d4c970c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,6 +2144,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,
@@ -2151,7 +2152,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_confl_update_deleted.pl b/src/test/subscription/t/035_confl_update_deleted.pl
index b60092ec24..d91bae7d9b 100644
--- a/src/test/subscription/t/035_confl_update_deleted.pl
+++ b/src/test/subscription/t/035_confl_update_deleted.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -116,10 +118,22 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
my $last_xid = $node_A->safe_psql('postgres',
"SELECT xid from pg_last_committed_xact();");
--
2.30.0.windows.2
On Thursday, January 2, 2025 6:34 PM vignesh C <vignesh21@gmail.com> wrote:
Few suggestions:
1) If we have a subscription with detect_update_deleted option and we
try to upgrade it with default settings(in case dba forgot to set
track_commit_timestamp), the upgrade will fail after doing a lot of
steps like that mentioned in ok below:
Setting locale and encoding for new cluster ok
Analyzing all rows in the new cluster ok
Freezing all rows in the new cluster ok
Deleting files from new pg_xact ok
Copying old pg_xact to new server ok
Setting oldest XID for new cluster ok
Setting next transaction ID and epoch for new cluster ok
Deleting files from new pg_multixact/offsets ok
Copying old pg_multixact/offsets to new server ok
Deleting files from new pg_multixact/members ok
Copying old pg_multixact/members to new server ok
Setting next multixact ID and offset for new cluster ok
Resetting WAL archives ok
Setting frozenxid and minmxid counters in new cluster ok
Restoring global objects in the new cluster ok
Restoring database schemas in the new cluster
postgres
*failure*We should detect this at an earlier point somewhere like in
check_new_cluster_subscription_configuration and throw an error from
there.2) Also should we include an additional slot for the
pg_conflict_detection slot while checking max_replication_slots.
Though this error will occur after the upgrade is completed, it may be
better to include the slot during upgrade itself so that the DBA need
not handle this error separately after the upgrade is completed.
Thanks for the comments!
I added the suggested changes but didn't add more tests to verify each error
message in this version, because it seems a rare case to me, so I am not sure
if it's worth increasing the testing time for these errors. But I am OK to add
if people think it's worth the effort and I will also test this locally.
Best Regards,
Hou zj
On Thursday, January 2, 2025 2:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Sounds reasonable but OTOH, all other places that create physical
slots (which we are doing here) don't use this trick. So, don't they
need similar reliability?
I have not figured the reason for existing physical slots' handling,
but will think more.
Also, add some comments as to why we are
initially creating the RS_EPHEMERAL slot as we have at other places.
Added.
Few other comments on 0003 ======================= 1. + if (sublist) + { + bool updated; + + if (!can_advance_xmin) + xmin = InvalidFullTransactionId; + + updated = advance_conflict_slot_xmin(xmin);How will it help to try advancing slot_xmin when xmin is invalid?
It was intended to create the slot without updating the xmin in this case,
but the function name seems misleading. So, I will think more on this and
modify it in next version because it may also be affected by the discussion
in [1]/messages/by-id/CAA4eK1Li8XLJ5f-pYvPJ8pXxyA3G-QsyBLNzHY940amF7jm=3A@mail.gmail.com.
2.
@@ -1167,14 +1181,43 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;if (!sub->enabled) + { + can_advance_xmin = false;In ApplyLauncherMain(), if one of the subscriptions is disabled (say
the last one in sublist), then can_advance_xmin will become false in
the above code. Now, later, as quoted in comment-1, the patch
overrides xmin to InvalidFullTransactionId if can_advance_xmin is
false. Won't that lead to the wrong computation of xmin?
advance_conflict_slot_xmin() would skip updating the slot.xmin
if the input value is invalid. But I will think how to improve this
in next version.
3. + 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;Can't we use MyReplicationSlot instead of introducing a new boolean
slot_maybe_exist?In any case, how does the above code deal with the case where the
launcher is restarted for some reason and there is no subscription
after that? Will it be possible to drop the slot in that case?
Since the initial value of slot_maybe_exist is true, so I think the launcher would
always check the slot once and drop the slot if not needed even if the
launcher restarted.
[1]: /messages/by-id/CAA4eK1Li8XLJ5f-pYvPJ8pXxyA3G-QsyBLNzHY940amF7jm=3A@mail.gmail.com
Best Regards,
Hou zj
On Tue, 7 Jan 2025 at 18:04, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, January 3, 2025 1:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 25, 2024 at 8:13 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the new version patch set which addressed all other comments.
Some more miscellaneous comments:
Thanks for the comments!
============================= 1. @@ -1431,9 +1431,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. @@ -1536,7 +1536,7 @@ RecordTransactionCommit(void) */ if (markXidCommitted) { - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION();The comments related to this change should be updated in EndPrepare()
and RecordTransactionCommitPrepared(). They still refer to the
DELAY_CHKPT_START flag. We should update the comments explaining why
a
similar change is not required for prepare or commit_prepare, if there
is one.After considering more, I think we need to use the new flag in
RecordTransactionCommitPrepared() as well, because it is assigned a commit
timestamp and would be replicated as normal transaction if sub's two_phase is
not enabled.3. +FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot, + TransactionId *delete_xid, + RepOriginId *delete_origin, + TimestampTz *delete_time) ... ... + /* 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); + + if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD) + dead = true; + + LockBuffer(buf, BUFFER_LOCK_UNLOCK); + + if (!dead) + continue;Why do we need to check only for HEAPTUPLE_RECENTLY_DEAD and not
HEAPTUPLE_DEAD? IIUC, we came here because we couldn't find the live
tuple, now whether the tuple is DEAD or RECENTLY_DEAD, why should it
matter to detect update_delete conflict?The HEAPTUPLE_DEAD could indicate tuples whose inserting transaction was
aborted, in which case we could not get the commit timestamp or origin for the
transaction. Or it could indicate tuples deleted by a transaction older than
oldestXmin(we would take the new replication slot's xmin into account when
computing this value), which means any subsequent transaction would have commit
timestamp later than that old delete transaction, so I think it's OK to ignore
this dead tuple and even detect update_missing because the resolution is to
apply the subsequent UPDATEs anyway (assuming we are using last update win
strategy). I added some comments along these lines in the patch.5. + + <varlistentry id="sql-createsubscription-params-with-detect-update-deleted"> + <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term> + <listitem> + <para> + Specifies whether the detection of <xref linkend="conflict-update-deleted"/> + is enabled. The default is <literal>false</literal>. If set to + true, the dead tuples on the subscriber that are still useful for + detecting <xref linkend="conflict-update-deleted"/> + are retained,One of the purposes of retaining dead tuples is to detect
update_delete conflict. But, I also see the following in 0001's commit
message: "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."
which indicates there are other cases where retaining dead tuples can
help. So, I was thinking about whether to name this new option as
retain_dead_tuples or something along those lines?I used the retain_conflict_info in this version as it looks more general and we
are already using similar name in patch(RetainConflictInfoData), but we can
change it later if people have better ideas.Attached the V19 patch which addressed comments in [1][2][3][4][5][6][7].
Few comments:
1) All other options are ordered, we can mention retain_conflict_info
after password_required to keep it consistent, I think it got
misplaced because of the name change from detect_update_deleted to
retain_conflict_info:
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index bbd08770c3..9d07fbf07a 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,9 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> 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");
+ COMPLETE_WITH("binary", "retain_conflict_info",
"disable_on_error",
+ "failover", "origin",
"password_required",
+ "run_as_owner", "slot_name",
"streaming",
+ "synchronous_commit", "two_phase");
2) Similarly here too:
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
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");
+ "retain_conflict_info",
"disable_on_error", "enabled",
3) Now that the option detect_update_deleted is changed to
retain_conflict_info, we can change this to "Retain conflict info":
+ if (pset.sversion >= 180000)
+ appendPQExpBuffer(&buf,
+ ",
subretainconflictinfo AS \"%s\"\n",
+
gettext_noop("Detect update deleted"));
4) The corresponding test changes also should be updated:
+++ 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 | Detect update deleted |
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
5) This part of code is not very easy to understand that it is done
for handling wrap around, could we add some comments here:
+ 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();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /* Compute the epoch of the oldest_running_xid */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
Regards,
Vignesh
On Tue, Jan 7, 2025 at 6:04 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attached the V19 patch which addressed comments in [1][2][3][4][5][6][7].
Here are a couple of initial review comments on v19 patch set:
1) The subscription option 'retain_conflict_info' remains set to
"true" for a subscription even after restarting the server with
'track_commit_timestamp=off', which can lead to incorrect behavior.
Steps to reproduce:
1. Start the server with 'track_commit_timestamp=ON'.
2. Create a subscription with (retain_conflict_info=ON).
3. Restart the server with 'track_commit_timestamp=OFF'.
- The apply worker starts successfully, and the subscription retains
'retain_conflict_info=true'. However, in this scenario, the
update_deleted conflict detection will not function correctly without
'track_commit_timestamp'.
```
postgres=# show track_commit_timestamp;
track_commit_timestamp
------------------------
off
(1 row)
postgres=# select subname, subretainconflictinfo from pg_subscription;
subname | subretainconflictinfo
---------+-----------------------
sub21 | t
sub22 | t
```
2) With the new parameter name change to "retain_conflict_info", the
error message for both the 'CREATE SUBSCRIPTION' and 'ALTER
SUBSCRIPTION' commands needs to be updated accordingly.
postgres=# create subscription sub11 connection 'dbname=postgres'
publication pub1 with (retain_conflict_info=on);
ERROR: detecting update_deleted conflicts requires
"track_commit_timestamp" to be enabled
postgres=# alter subscription sub12 set (retain_conflict_info=on);
ERROR: detecting update_deleted conflicts requires
"track_commit_timestamp" to be enabled
- Change the message to something similar - "retaining conflict info
requires "track_commit_timestamp" to be enabled".
--
Thanks,
Nisha
On Tue, Jan 7, 2025 at 2:49 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Jan 6, 2025 at 4:52 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Friday, January 3, 2025 2:36 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
+ /* + * The changes made by this and later transactions are still non-removable + * to allow for the detection of update_deleted conflicts when applying + * changes in this logical replication worker. + * + * Note that this info cannot directly protect dead tuples from being + * prematurely frozen or removed. The logical replication launcher + * asynchronously collects this info to determine whether to advance the + * xmin value of the replication slot. + * + * Therefore, FullTransactionId that includes both the transaction ID and + * its epoch is used here instead of a single Transaction ID. This is + * critical because without considering the epoch, the transaction ID + * alone may appear as if it is in the future due to transaction ID + * wraparound. + */ + FullTransactionId oldest_nonremovable_xid;The last paragraph of the comment mentions that we need to use
FullTransactionId to properly compare XIDs even after the XID wraparound
happens. But once we set the oldest-nonremovable-xid it prevents XIDs from
being wraparound, no? I mean that workers'
oldest-nonremovable-xid values and slot's non-removal-xid (i.e., its
xmin) are never away from more than 2^31 XIDs.I think the issue is that the launcher may create the replication slot after
the apply worker has already set the 'oldest_nonremovable_xid' because the
launcher are doing that asynchronously. So, Before the slot is created, there's
a window where transaction IDs might wrap around. If initially the apply worker
has computed a candidate_xid (755) and the xid wraparound before the launcher
creates the slot, causing the new current xid to be (740), then the old
candidate_xid(755) looks like a xid in the future, and the launcher could
advance the xmin to 755 which cause the dead tuples to be removed prematurely.
(We are trying to reproduce this to ensure that it's a real issue and will
share after finishing)We thought of another approach, which is to create/drop this slot first as
soon as one enables/disables detect_update_deleted (E.g. create/drop slot
during DDL). But it seems complicate to control the concurrent slot
create/drop. For example, if one backend A enables detect_update_deteled, it
will create a slot. But if another backend B is disabling the
detect_update_deteled at the same time, then the newly created slot may be
dropped by backend B. I thought about checking the number of subscriptions that
enables detect_update_deteled before dropping the slot in backend B, but the
subscription changes caused by backend A may not visable yet (e.g. not
committed yet).This means that for the transaction whose changes are not yet visible,
we may have already created the slot and the backend B would end up
dropping it. Is it possible that during the change of this new option
via DDL, we take AccessExclusiveLock on pg_subscription as we do in
DropSubscription() to ensure that concurrent transactions can't drop
the slot? Will that help in solving the above scenario?
If we create/stop the slot during DDL, how do we support rollback DDLs?
The second idea could be that each worker first checks whether a slot
exists along with a subscription flag (new option). Checking the
existence of a slot each time would be costly, so we somehow need to
cache it. But if we do that then we need to invent some cache
invalidation mechanism for the slot. I am not sure if we can design a
race-free mechanism for that. I mean we need to think of a solution
for race conditions between the launcher and apply workers to ensure
that after dropping the slot, launcher doesn't recreate the slot (say
if some subscription enables this option) before all the workers can
clear their existing values of oldest_nonremovable_xid.The third idea to avoid the race condition could be that in the
function InitializeLogRepWorker() after CommitTransactionCommand(), we
check if the retain_dead_tuples flag is true for MySubscription then
we check whether the system slot exists. If exits then go ahead,
otherwise, wait till the slot is created. It could be some additional
cycles during worker start up but it is a one-time effort and that too
only when the flag is set. In addition to this, we anyway need to
create the slot in the launcher before launching the workers, and
after re-reading the subscription, the change in retain_dead_tuples
flag (off->on) should cause the worker restart.Now, in the third idea, the issue can still arise if, after waiting
for the slot to be created, the user sets the retain_dead_tuples to
false and back to true again immediately. Because the launcher may
have noticed the "retain_dead_tuples=false" operation and dropped the
slot, while the apply worker has not noticed and still holds an old
candidate_xid. The xid may wraparound in this window before setting
the retain_dead_tuples back to true. And, the apply worker would not
restart because after it calls maybe_reread_subscription(), the
retain_dead_tuples would have been set back to true again. Again, to
avoid this race condition, the launcher can wait for each worker to
reset the oldest_nonremovamble_xid before dropping the slot.Even after doing the above, the third idea could still have another
race condition:
1. The launcher creates the replication slot and starts a worker with
retain_dead_tuples = true, the worker is waiting for publish status
and has not set oldest_nonremovable_xid.
2. The user set the option retain_dead_tuples to false, the launcher
noticed that and drop the replication slot.
3. The worker received the status and set oldest_nonremovable_xid to a
valid value (say 750).
4. Xid wraparound happened at this point and say new_available_xid becomes 740
5. User set retain_dead_tuples = true again.After the above steps, the apply worker holds an old
oldest_nonremovable_xid (750) and will not restart if it does not call
maybe_reread_subscription() before step 5. So, such a case can again
create a problem of incorrect slot->xmin value. We can probably try to
find some way to avoid this race condition as well but I haven't
thought more about this as this idea sounds a bit risky and bug-prone
to me.Among the above ideas, the first idea of taking AccessExclusiveLock on
pg_subscription sounds safest to me. I haven't evaluated the changes
for the first approach so I could be missing something that makes it
difficult to achieve but I think it is worth investigating unless we
have better ideas or we think that the current approach used in patch
to use FullTransactionId is okay.
Thank you for considering some ideas. As I mentioned above, we might
need to consider a case like where 'CREATE SUBSCRIPTION ..
(retain_conflict_info = true)' is rolled back. Having said that, this
comment is just for simplifying the logic. If using TransactionId
instead makes other parts complex, it would not make sense. I'm okay
with leaving this part and improving the comment for
oldest_nonremovable_xid, say, by mentioning that there is a window for
XID wraparound happening between workers computing their
oldst_nonremovable_xid and pg_conflict_detection slot being created.
BTW while reviewing the code, I realized that changing
retain_conflict_info value doesn't have the worker relaunch and we
don't clear the worker's oldest_nonremovable_xid value in this case.
Is it okay? I'm concerned about a case like where
RetainConflictInfoPhase state transition is paused by disabling
retain_conflict_info and resume by re-enabling it with an old
RetainConflictInfoData value.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wed, Jan 8, 2025 at 2:15 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Tue, Jan 7, 2025 at 2:49 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
We thought of another approach, which is to create/drop this slot first as
soon as one enables/disables detect_update_deleted (E.g. create/drop slot
during DDL). But it seems complicate to control the concurrent slot
create/drop. For example, if one backend A enables detect_update_deteled, it
will create a slot. But if another backend B is disabling the
detect_update_deteled at the same time, then the newly created slot may be
dropped by backend B. I thought about checking the number of subscriptions that
enables detect_update_deteled before dropping the slot in backend B, but the
subscription changes caused by backend A may not visable yet (e.g. not
committed yet).This means that for the transaction whose changes are not yet visible,
we may have already created the slot and the backend B would end up
dropping it. Is it possible that during the change of this new option
via DDL, we take AccessExclusiveLock on pg_subscription as we do in
DropSubscription() to ensure that concurrent transactions can't drop
the slot? Will that help in solving the above scenario?If we create/stop the slot during DDL, how do we support rollback DDLs?
We will prevent changing this setting in a transaction block as we
already do for slot related case. See use of
PreventInTransactionBlock() in subscriptioncmds.c.
Thank you for considering some ideas. As I mentioned above, we might
need to consider a case like where 'CREATE SUBSCRIPTION ..
(retain_conflict_info = true)' is rolled back. Having said that, this
comment is just for simplifying the logic. If using TransactionId
instead makes other parts complex, it would not make sense. I'm okay
with leaving this part and improving the comment for
oldest_nonremovable_xid, say, by mentioning that there is a window for
XID wraparound happening between workers computing their
oldst_nonremovable_xid and pg_conflict_detection slot being created.
Fair enough. Let us see what you think about my above response first.
--
With Regards,
Amit Kapila.
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
Here is further performance test analysis with v16 patch-set.
In the test scenarios already shared on -hackers [1], where pgbench was run only on the publisher node in a pub-sub setup, no performance degradation was observed on either node.
In contrast, when pgbench was run only on the subscriber side with detect_update_deleted=on [2], the TPS performance was reduced due to dead tuple accumulation. This performance drop depended on the wal_receiver_status_interval—larger intervals resulted in more dead tuple accumulation on the subscriber node. However, after the improvement in patch v16-0002, which dynamically tunes the status request, the default TPS reduction was limited to only 1%.
We performed more benchmarks with the v16-patches where pgbench was run on both the publisher and subscriber, focusing on TPS performance. To summarize the key observations:
- No performance impact on the publisher as dead tuple accumulation does not occur on the publisher.
Nice. It means that frequently getting in-commit-phase transactions by
the subscriber didn't have a negative impact on the publisher's
performance.
- The performance is reduced on the subscriber side (TPS reduction (~50%) [3] ) due to dead tuple retention for the conflict detection when detect_update_deleted=on.
- Performance reduction happens only on the subscriber side, as workload on the publisher is pretty high and the apply workers must wait for the amount of transactions with earlier timestamps to be applied and flushed before advancing the non-removable XID to remove dead tuples.
Assuming that the performance dip happened due to dead tuple retention
for the conflict detection, would TPS on other databases also be
affected?
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples created
by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?
Test Run:
- Ran pgbench(read-write) on both the publisher and the subscriber with 30 clients for a duration of 120 seconds, collecting data over 5 runs.
- Note that pgbench was running for different tables on pub and sub.
(The scripts used for test "case1-2_measure.sh" and case1-2_setup.sh" are attached).
Results:
Run# pub TPS sub TPS
1 32209 13704
2 32378 13684
3 32720 13680
4 31483 13681
5 31773 13813
median 32209 13684
regression 7% -53%
What was the TPS on the subscriber when detect_update_deleted = false?
And how much were the tables bloated compared to when
detect_update_deleted = false?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
Here is further performance test analysis with v16 patch-set.
In the test scenarios already shared on -hackers [1], where pgbench was run only on the publisher node in a pub-sub setup, no performance degradation was observed on either node.
In contrast, when pgbench was run only on the subscriber side with detect_update_deleted=on [2], the TPS performance was reduced due to dead tuple accumulation. This performance drop depended on the wal_receiver_status_interval—larger intervals resulted in more dead tuple accumulation on the subscriber node. However, after the improvement in patch v16-0002, which dynamically tunes the status request, the default TPS reduction was limited to only 1%.
We performed more benchmarks with the v16-patches where pgbench was run on both the publisher and subscriber, focusing on TPS performance. To summarize the key observations:
- No performance impact on the publisher as dead tuple accumulation does not occur on the publisher.
Nice. It means that frequently getting in-commit-phase transactions by
the subscriber didn't have a negative impact on the publisher's
performance.- The performance is reduced on the subscriber side (TPS reduction (~50%) [3] ) due to dead tuple retention for the conflict detection when detect_update_deleted=on.
- Performance reduction happens only on the subscriber side, as workload on the publisher is pretty high and the apply workers must wait for the amount of transactions with earlier timestamps to be applied and flushed before advancing the non-removable XID to remove dead tuples.
Assuming that the performance dip happened due to dead tuple retention
for the conflict detection, would TPS on other databases also be
affected?
As we use slot->xmin to retain dead tuples, shouldn't the impact be
global (means on all databases)? Or, maybe I am missing something.
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples created
by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?
I think hot-pruning mechanism during the update operation will remove
dead tuples even when autovacuum is disabled.
--
With Regards,
Amit Kapila.
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
Here is further performance test analysis with v16 patch-set.
In the test scenarios already shared on -hackers [1], where pgbench was run only on the publisher node in a pub-sub setup, no performance degradation was observed on either node.
In contrast, when pgbench was run only on the subscriber side with detect_update_deleted=on [2], the TPS performance was reduced due to dead tuple accumulation. This performance drop depended on the wal_receiver_status_interval—larger intervals resulted in more dead tuple accumulation on the subscriber node. However, after the improvement in patch v16-0002, which dynamically tunes the status request, the default TPS reduction was limited to only 1%.
We performed more benchmarks with the v16-patches where pgbench was run on both the publisher and subscriber, focusing on TPS performance. To summarize the key observations:
- No performance impact on the publisher as dead tuple accumulation does not occur on the publisher.
Nice. It means that frequently getting in-commit-phase transactions by
the subscriber didn't have a negative impact on the publisher's
performance.- The performance is reduced on the subscriber side (TPS reduction (~50%) [3] ) due to dead tuple retention for the conflict detection when detect_update_deleted=on.
- Performance reduction happens only on the subscriber side, as workload on the publisher is pretty high and the apply workers must wait for the amount of transactions with earlier timestamps to be applied and flushed before advancing the non-removable XID to remove dead tuples.
Assuming that the performance dip happened due to dead tuple retention
for the conflict detection, would TPS on other databases also be
affected?As we use slot->xmin to retain dead tuples, shouldn't the impact be
global (means on all databases)?
I think so too.
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples created
by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will remove
dead tuples even when autovacuum is disabled.
True, but why did it disable autovacuum? It seems that
case1-2_setup.sh doesn't specify fillfactor, which makes hot-updates
less likely to happen.
I understand that a certain performance dip happens due to dead tuple
retention, which is fine, but I'm surprised that the TPS decreased by
50% within 120 seconds. The TPS goes even worse for a longer test? I
did a quick benchmark where I completely disabled removing dead tuples
(by autovacuum=off and a logical slot) and ran pgbench but I didn't
see such a precipitous dip.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wednesday, January 8, 2025 6:33 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond
<nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples created
by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will remove
dead tuples even when autovacuum is disabled.True, but why did it disable autovacuum? It seems that case1-2_setup.sh
doesn't specify fillfactor, which makes hot-updates less likely to happen.
IIUC, we disable autovacuum as a general practice in read-write tests for
stable TPS numbers.
I understand that a certain performance dip happens due to dead tuple
retention, which is fine, but I'm surprised that the TPS decreased by 50% within
120 seconds. The TPS goes even worse for a longer test?
We will try to increase the time and run the test again.
I did a quick
benchmark where I completely disabled removing dead tuples (by
autovacuum=off and a logical slot) and ran pgbench but I didn't see such a
precipitous dip.
I think a logical slot only retain the dead tuples on system catalog,
so the TPS on user table would not be affected that much.
Best Regards,
Hou zj
On Tue, 7 Jan 2025 at 18:04, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, January 3, 2025 1:53 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Dec 25, 2024 at 8:13 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the new version patch set which addressed all other comments.
Some more miscellaneous comments:
Thanks for the comments!
============================= 1. @@ -1431,9 +1431,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. @@ -1536,7 +1536,7 @@ RecordTransactionCommit(void) */ if (markXidCommitted) { - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION();The comments related to this change should be updated in EndPrepare()
and RecordTransactionCommitPrepared(). They still refer to the
DELAY_CHKPT_START flag. We should update the comments explaining why
a
similar change is not required for prepare or commit_prepare, if there
is one.After considering more, I think we need to use the new flag in
RecordTransactionCommitPrepared() as well, because it is assigned a commit
timestamp and would be replicated as normal transaction if sub's two_phase is
not enabled.3. +FindMostRecentlyDeletedTupleInfo(Relation rel, TupleTableSlot *searchslot, + TransactionId *delete_xid, + RepOriginId *delete_origin, + TimestampTz *delete_time) ... ... + /* 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); + + if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf) == HEAPTUPLE_RECENTLY_DEAD) + dead = true; + + LockBuffer(buf, BUFFER_LOCK_UNLOCK); + + if (!dead) + continue;Why do we need to check only for HEAPTUPLE_RECENTLY_DEAD and not
HEAPTUPLE_DEAD? IIUC, we came here because we couldn't find the live
tuple, now whether the tuple is DEAD or RECENTLY_DEAD, why should it
matter to detect update_delete conflict?The HEAPTUPLE_DEAD could indicate tuples whose inserting transaction was
aborted, in which case we could not get the commit timestamp or origin for the
transaction. Or it could indicate tuples deleted by a transaction older than
oldestXmin(we would take the new replication slot's xmin into account when
computing this value), which means any subsequent transaction would have commit
timestamp later than that old delete transaction, so I think it's OK to ignore
this dead tuple and even detect update_missing because the resolution is to
apply the subsequent UPDATEs anyway (assuming we are using last update win
strategy). I added some comments along these lines in the patch.5. + + <varlistentry id="sql-createsubscription-params-with-detect-update-deleted"> + <term><literal>detect_update_deleted</literal> (<type>boolean</type>)</term> + <listitem> + <para> + Specifies whether the detection of <xref linkend="conflict-update-deleted"/> + is enabled. The default is <literal>false</literal>. If set to + true, the dead tuples on the subscriber that are still useful for + detecting <xref linkend="conflict-update-deleted"/> + are retained,One of the purposes of retaining dead tuples is to detect
update_delete conflict. But, I also see the following in 0001's commit
message: "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."
which indicates there are other cases where retaining dead tuples can
help. So, I was thinking about whether to name this new option as
retain_dead_tuples or something along those lines?I used the retain_conflict_info in this version as it looks more general and we
are already using similar name in patch(RetainConflictInfoData), but we can
change it later if people have better ideas.Attached the V19 patch which addressed comments in [1][2][3][4][5][6][7].
Consider a LR setup with retain_conflict_info=true for a table t1:
Publisher:
insert into t1 values(1);
-- Have a open transaction before delete operation in subscriber
begin;
Subscriber:
-- delete the record that was replicated
delete from t1;
-- Now commit the transaction in publisher
Publisher:
update t1 set c1 = 2;
commit;
In normal case update_deleted conflict is detected
2025-01-08 15:41:38.529 IST [112744] LOG: conflict detected on
relation "public.t1": conflict=update_deleted
2025-01-08 15:41:38.529 IST [112744] DETAIL: The row to be updated
was deleted locally in transaction 751 at 2025-01-08
15:41:29.811566+05:30.
Remote tuple (2); replica identity full (1).
2025-01-08 15:41:38.529 IST [112744] CONTEXT: processing remote data
for replication origin "pg_16387" during message type "UPDATE" for
replication target relation "public.t1" in transaction 747, finished
at 0/16FBCA0
Now execute the same above case by having a presetup to consume all
the replication slots in the system by executing
pg_create_logical_replication_slot before the subscription is created,
in this case the conflict is not detected correctly.
2025-01-08 15:39:17.931 IST [112551] LOG: conflict detected on
relation "public.t1": conflict=update_missing
2025-01-08 15:39:17.931 IST [112551] DETAIL: Could not find the row
to be updated.
Remote tuple (2); replica identity full (1).
2025-01-08 15:39:17.931 IST [112551] CONTEXT: processing remote data
for replication origin "pg_16387" during message type "UPDATE" for
replication target relation "public.t1" in transaction 747, finished
at 0/16FBC68
2025-01-08 15:39:18.266 IST [112582] ERROR: all replication slots are in use
2025-01-08 15:39:18.266 IST [112582] HINT: Free one or increase
"max_replication_slots".
This is because even though we say create subscription is successful,
the launcher has not yet created the replication slot.
There are few observations from this test:
1) Create subscription does not wait for the slot to be created by the
launcher and starts applying the changes. Should create a subscription
wait till the slot is created by the launcher process.
2) Currently launcher is exiting continuously and trying to create
replication slots. Should the launcher wait for
wal_retrieve_retry_interval configuration before trying to create the
slot instead of filling the logs continuously.
3) If we try to create a similar subscription with
retain_conflict_info and disable_on_error option and there is an error
in replication slot creation, currently the subscription does not get
disabled. Should we consider disable_on_error for these cases and
disable the subscription if we are not able to create the slots.
Regards,
Vignesh
On Wed, Jan 8, 2025 at 3:00 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wednesday, January 8, 2025 6:33 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond
<nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples created
by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will remove
dead tuples even when autovacuum is disabled.True, but why did it disable autovacuum? It seems that case1-2_setup.sh
doesn't specify fillfactor, which makes hot-updates less likely to happen.IIUC, we disable autovacuum as a general practice in read-write tests for
stable TPS numbers.
Okay. TBH I'm not sure what we can say with these results. At a
glance, in a typical bi-directional-like setup, we can interpret
these results as that if users turn retain_conflict_info on the TPS
goes 50% down. But I'm not sure this 50% dip is the worst case that
users possibly face. It could be better in practice thanks to
autovacuum, or it also could go even worse due to further bloats if we
run the test longer.
Suppose that users had 50% performance dip due to dead tuple retention
for update_deleted detection, is there any way for users to improve
the situation? For example, trying to advance slot.xmin more
frequently might help to reduce dead tuple accumulation. I think it
would be good if we could have a way to balance between the publisher
performance and the subscriber performance.
In test case 3, we observed a -53% performance dip, which is worse
than the results of test case 5 with wal_receiver_status_interval =
100s. Given that in test case 5 with wal_receiver_status_interval =
100s we cannot remove dead tuples for the most of the whole 120s test
time, probably we could not remove dead tuples for a long time also in
test case 3. I expected that the apply worker gets remote transaction
XIDs and tries to advance slot.xmin more frequently, so this
performance dip surprised me. I would like to know how many times the
apply worker gets remote transaction XIDs and succeeds in advance
slot.xmin during the test.
I understand that a certain performance dip happens due to dead tuple
retention, which is fine, but I'm surprised that the TPS decreased by 50% within
120 seconds. The TPS goes even worse for a longer test?We will try to increase the time and run the test again.
I did a quick
benchmark where I completely disabled removing dead tuples (by
autovacuum=off and a logical slot) and ran pgbench but I didn't see such a
precipitous dip.I think a logical slot only retain the dead tuples on system catalog,
so the TPS on user table would not be affected that much.
You're right, I missed it.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Thursday, January 9, 2025 9:48 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 3:00 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Wednesday, January 8, 2025 6:33 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond
<nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples
created by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will
remove dead tuples even when autovacuum is disabled.True, but why did it disable autovacuum? It seems that
case1-2_setup.sh doesn't specify fillfactor, which makes hot-updates lesslikely to happen.
IIUC, we disable autovacuum as a general practice in read-write tests
for stable TPS numbers.Okay. TBH I'm not sure what we can say with these results. At a glance, in a
typical bi-directional-like setup, we can interpret these results as that if
users turn retain_conflict_info on the TPS goes 50% down. But I'm not sure
this 50% dip is the worst case that users possibly face. It could be better in
practice thanks to autovacuum, or it also could go even worse due to further
bloats if we run the test longer.
I think it shouldn't go worse because ideally the amount of bloat would not
increase beyond what we see here due to this patch unless there is some
misconfiguration that leads to one of the node not working properly (say it is
down). However, my colleague is running longer tests and we will share the
results soon.
Suppose that users had 50% performance dip due to dead tuple retention for
update_deleted detection, is there any way for users to improve the situation?
For example, trying to advance slot.xmin more frequently might help to reduce
dead tuple accumulation. I think it would be good if we could have a way to
balance between the publisher performance and the subscriber performance.
AFAICS, most of the time in each xid advancement is spent on waiting for the
target remote_lsn to be applied and flushed, so increasing the frequency could
not help. This can be proved to be reasonable in the testcase 4 shared by
Nisha[1]/messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com, in that test, we do not request a remote_lsn but simply wait for the
commit_ts of incoming transaction to exceed the candidate_xid_time, the
regression is still the same. I think it indicates that we indeed need to wait
for this amount of time before applying all the transactions that have earlier
commit timestamp. IOW, the performance impact on the subscriber side is a
reasonable behavior if we want to detect the update_deleted conflict reliably.
[1]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
Best Regards,
Hou zj
On Thursday, January 9, 2025 9:48 AM Masahiko Sawada <sawada.mshk@gmail.com>
Hi,
On Wed, Jan 8, 2025 at 3:00 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Wednesday, January 8, 2025 6:33 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond
<nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples
created by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will
remove dead tuples even when autovacuum is disabled.True, but why did it disable autovacuum? It seems that
case1-2_setup.sh doesn't specify fillfactor, which makes hot-updates lesslikely to happen.
IIUC, we disable autovacuum as a general practice in read-write tests
for stable TPS numbers.
...
In test case 3, we observed a -53% performance dip, which is worse than the
results of test case 5 with wal_receiver_status_interval = 100s. Given that
in test case 5 with wal_receiver_status_interval = 100s we cannot remove dead
tuples for the most of the whole 120s test time, probably we could not remove
dead tuples for a long time also in test case 3. I expected that the apply
worker gets remote transaction XIDs and tries to advance slot.xmin more
frequently, so this performance dip surprised me.
As noted in my previous email[1]/messages/by-id/OS0PR01MB57164C9A65F29875AE63F0BD94132@OS0PR01MB5716.jpnprd01.prod.outlook.com, the delay primarily occurs during the final
phase (RCI_WAIT_FOR_LOCAL_FLUSH), where we wait for concurrent transactions
from the publisher to be applied and flushed locally (e.g., last_flushpos <
data->remote_lsn). I think that the interval between each transaction ID
advancement is brief, the duration of each advancement itself is significant.
I would like to know how many times the apply worker gets remote transaction
XIDs and succeeds in advance slot.xmin during the test.
my colleague will collect and share the data soon.
[1]: /messages/by-id/OS0PR01MB57164C9A65F29875AE63F0BD94132@OS0PR01MB5716.jpnprd01.prod.outlook.com
Best Regards,
Hou zj
On Wed, Jan 8, 2025 at 2:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 8, 2025 at 2:15 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Tue, Jan 7, 2025 at 2:49 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
We thought of another approach, which is to create/drop this slot first as
soon as one enables/disables detect_update_deleted (E.g. create/drop slot
during DDL). But it seems complicate to control the concurrent slot
create/drop. For example, if one backend A enables detect_update_deteled, it
will create a slot. But if another backend B is disabling the
detect_update_deteled at the same time, then the newly created slot may be
dropped by backend B. I thought about checking the number of subscriptions that
enables detect_update_deteled before dropping the slot in backend B, but the
subscription changes caused by backend A may not visable yet (e.g. not
committed yet).This means that for the transaction whose changes are not yet visible,
we may have already created the slot and the backend B would end up
dropping it. Is it possible that during the change of this new option
via DDL, we take AccessExclusiveLock on pg_subscription as we do in
DropSubscription() to ensure that concurrent transactions can't drop
the slot? Will that help in solving the above scenario?If we create/stop the slot during DDL, how do we support rollback DDLs?
We will prevent changing this setting in a transaction block as we
already do for slot related case. See use of
PreventInTransactionBlock() in subscriptioncmds.c.
On further thinking, even if we prevent this command in a transaction
block, there is still a small chance of rollback. Say, we created the
slot as the last operation after making database changes, but still,
the transaction can fail in the commit code path. So, it is still not
bulletproof. However, we already create a remote_slot at the end of
CREATE SUBSCRIPTION, so, if by any chance the transaction fails in the
commit code path, we will end up having a dangling slot on the remote
node. The same can happen in the DROP SUBSCRIPTION code path as well.
We can follow that or the other option is to allow creation of the
slot by the backend and let drop be handled by the launcher which can
even take care of dangling slots. However, I feel it will be better to
give the responsibility to the launcher for creating and dropping the
slot as the patch is doing and use the FullTransactionId for each
worker. What do you think?
--
With Regards,
Amit Kapila.
On Wednesday, January 8, 2025 3:00 PM vignesh C <vignesh21@gmail.com> wrote:
On Tue, 7 Jan 2025 at 18:04, Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Attached the V19 patch which addressed comments in [1][2][3][4][5][6][7].
Few comments:
Thanks for the comments. Here is the V20 patch which addressed them and
comments in [1]/messages/by-id/CABdArM70XAQOK1GiXJC4gpXewGGYxixxVyhxCYXhVFDqnjLL4A@mail.gmail.com[2]/messages/by-id/CALDaNm3+x+ZRE9wynMofvc_ZxsLXKHpyS7LYxnP-a2=RQE-4uA@mail.gmail.com.
I also modified some comments in 0003 to be consistent with the new
option name.
[1]: /messages/by-id/CABdArM70XAQOK1GiXJC4gpXewGGYxixxVyhxCYXhVFDqnjLL4A@mail.gmail.com
[2]: /messages/by-id/CALDaNm3+x+ZRE9wynMofvc_ZxsLXKHpyS7LYxnP-a2=RQE-4uA@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v20-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v20-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 37312e226179579c15df52f10538682f44c1d842 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v20 2/5] 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 | 224 ++++++++++++++++++
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/slot.c | 34 ++-
src/include/replication/slot.h | 11 +-
10 files changed, 287 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 8683f0bdf53..6d5f4973c41 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4877,6 +4877,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581ae..feb026d0c5a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..01b37a5a3e1 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea206..d49ba78b13e 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..350c86540f8 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1119,6 +1129,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1160,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();
@@ -1167,14 +1182,49 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
+
+ /*
+ * 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 the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1257,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1311,153 @@ 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);
+ return;
+ }
+
+ /*
+ * Initially create persistent slot as ephemeral - that allows us to
+ * nicely handle errors during initialization because it'll get dropped if
+ * this transaction fails. We'll make it persistent after safely
+ * initializing the xmin of the slot.
+ */
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf0..64b43346a90 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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/slot.c b/src/backend/replication/slot.c
index b30e0473e1c..3a3884b3820 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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/slot.h b/src/include/replication/slot.h
index bf62b36ad07..ea0485efa2e 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.
*
@@ -261,7 +268,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);
--
2.30.0.windows.2
v20-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v20-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From ba5a0e0f6852390b5b53b1a262dfa87bcc1f2e24 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 9 Jan 2025 16:44:53 +0800
Subject: [PATCH v20 3/5] 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 | 46 +++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 ++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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, 449 insertions(+), 124 deletions(-)
create mode 100644 src/backend/replication/logical/launcher.c.rej
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cc6cf9bef09..2835a1e6cc7 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8041,6 +8041,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a083..bde698348f4 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2160,7 +2160,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 01b37a5a3e1..5d3fe3b7d02 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 7a595c84db9..cee60482b65 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 350c86540f8..b3b57daf196 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -158,6 +158,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);
@@ -1161,6 +1162,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();
@@ -1181,12 +1183,21 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * 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.
+ */
+ if (sub->retainconflictinfo)
{
- can_advance_xmin = false;
- continue;
+ retain_conflict_info = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1195,10 +1206,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;
@@ -1224,7 +1236,8 @@ ApplyLauncherMain(Datum main_arg)
* 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 (retain_conflict_info)
+ create_conflict_slot_if_not_exists();
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1262,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1276,12 +1289,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1312,7 +1327,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
@@ -1358,7 +1373,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1369,7 +1384,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1446,8 +1464,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 31570377a99..5dcd01e6864 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4102,6 +4102,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
@@ -4503,6 +4507,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 ||
@@ -4512,7 +4525,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 8f73a5df956..7940dc5803f 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1b..0be12d05c93 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index d5543fd62b0..92e0358e39e 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 6194b757d59..1d1e0dffb9e 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v20-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v20-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From a1dde7dfd02aed5886740d65eaa2936c203feb46 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v20 1/5] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 490 ++++++++++++++++++++-
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, 744 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1a..0dcd4afe4cb 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1182,7 +1182,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.
@@ -2356,7 +2360,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2380,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2393,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
@@ -2439,7 +2453,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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7aff..31570377a99 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,60 @@ 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 */
+
+ /*
+ * 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 +394,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 +441,17 @@ 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 inline bool can_advance_nonremovable_xid(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,
@@ -3573,6 +3646,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
@@ -3651,6 +3725,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);
@@ -3677,6 +3753,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3776,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);
@@ -3706,6 +3814,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)
{
/*
@@ -3740,6 +3853,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,
@@ -3803,6 +3923,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
@@ -3838,7 +3960,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;
@@ -3916,6 +4037,373 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * 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;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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;
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
+/*
+ * 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 a0782b1bbf6..1299ed1eb5d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 9f83ecf181f..2e202065670 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2475,6 +2475,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v20-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v20-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 57f1f117e8852f397c7e85f5dac83de64056e429 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v20 4/5] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 156 ++++++++++++++++++
2 files changed, 157 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..7a69c13486a
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,156 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
v20-0005-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v20-0005-Support-the-conflict-detection-for-update_delete.patchDownload
From e0964fb75856c3ff46b56976c208fdc39d772dc4 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v20 5/5] 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index bde698348f4..7e8c959a2d3 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d0d176cc54f..878e955e627 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index cee60482b65..456e6835dbf 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,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 e3e4e41ac38..f08e7276054 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 5dcd01e6864..bea9895363b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2742,6 +2742,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2758,15 +2761,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2777,7 +2776,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2796,19 +2795,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3129,9 +3137,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3140,20 +3148,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3162,7 +3179,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3173,7 +3190,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 3245f3a8d8a..80e0c8150f1 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2087,7 +2087,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2109,17 +2109,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index b37e8a6f882..e6c2f1d3425 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 f8a8d03e533..83ccfd1b504 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"
@@ -670,7 +671,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 37454dc9513..91224b6ea60 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 3014d047fef..2f5d4c970cd 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,6 +2144,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,
@@ -2151,7 +2152,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 7a69c13486a..65e4033cb5a 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -113,6 +118,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -120,6 +132,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
On Wednesday, January 8, 2025 7:03 PM vignesh C <vignesh21@gmail.com> wrote:
Hi,
Consider a LR setup with retain_conflict_info=true for a table t1:
Publisher:
insert into t1 values(1);
-- Have a open transaction before delete operation in subscriber begin;Subscriber:
-- delete the record that was replicated delete from t1;-- Now commit the transaction in publisher
Publisher:
update t1 set c1 = 2;
commit;In normal case update_deleted conflict is detected
2025-01-08 15:41:38.529 IST [112744] LOG: conflict detected on relation
"public.t1": conflict=update_deleted
2025-01-08 15:41:38.529 IST [112744] DETAIL: The row to be updated was
deleted locally in transaction 751 at 2025-01-08 15:41:29.811566+05:30.
Remote tuple (2); replica identity full (1).
2025-01-08 15:41:38.529 IST [112744] CONTEXT: processing remote data for
replication origin "pg_16387" during message type "UPDATE" for replication
target relation "public.t1" in transaction 747, finished at 0/16FBCA0Now execute the same above case by having a presetup to consume all the
replication slots in the system by executing pg_create_logical_replication_slot
before the subscription is created, in this case the conflict is not detected
correctly.
2025-01-08 15:39:17.931 IST [112551] LOG: conflict detected on relation
"public.t1": conflict=update_missing
2025-01-08 15:39:17.931 IST [112551] DETAIL: Could not find the row to be
updated.
Remote tuple (2); replica identity full (1).
2025-01-08 15:39:17.931 IST [112551] CONTEXT: processing remote data for
replication origin "pg_16387" during message type "UPDATE" for replication
target relation "public.t1" in transaction 747, finished at 0/16FBC68
2025-01-08 15:39:18.266 IST [112582] ERROR: all replication slots are in use
2025-01-08 15:39:18.266 IST [112582] HINT: Free one or increase
"max_replication_slots".This is because even though we say create subscription is successful, the
launcher has not yet created the replication slot.
I think some detection miss in the beginning after enabling the option is
acceptable. Because even if we let the launcher to create the slot before
starting workers, some dead tuples could have been already removed during this
period, so update_missing could still be detected. I have added some documents
to clarify that the information can be safely retained only after the slot is
created.
There are few observations from this test:
1) Create subscription does not wait for the slot to be created by the launcher
and starts applying the changes. Should create a subscription wait till the slot
is created by the launcher process.
I think the DDL could not wait for the slot creation, because the launcher would
not create the slot until the DDL is committed. Instead, I have changed the
code to create the slot before starting workers, so that at least the worker
would not unnecessarily maintain the oldest non-removable xid.
2) Currently launcher is exiting continuously and trying to create replication
slots. Should the launcher wait for wal_retrieve_retry_interval configuration
before trying to create the slot instead of filling the logs continuously.
Since the launcher already have a 5s (bgw_restart_time) restart interval, I
feel it would not consume the too much resources in this case.
3) If we try to create a similar subscription with retain_conflict_info and
disable_on_error option and there is an error in replication slot creation,
currently the subscription does not get disabled. Should we consider
disable_on_error for these cases and disable the subscription if we are not able
to create the slots.
Currently, since only ERRORs in apply worker would trigger disable_on_error, I
am not sure if It's worth the effort to teach the apply to catch launcher's
error because it doesn't seem like a common scenario.
Best Regards,
Hou zj
On Wednesday, January 8, 2025 3:49 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
On Tue, Jan 7, 2025 at 6:04 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Attached the V19 patch which addressed comments in [1][2][3][4][5][6][7].
Here are a couple of initial review comments on v19 patch set:
1) The subscription option 'retain_conflict_info' remains set to "true" for a
subscription even after restarting the server with
'track_commit_timestamp=off', which can lead to incorrect behavior.
Steps to reproduce:
1. Start the server with 'track_commit_timestamp=ON'.
2. Create a subscription with (retain_conflict_info=ON).
3. Restart the server with 'track_commit_timestamp=OFF'.- The apply worker starts successfully, and the subscription retains
'retain_conflict_info=true'. However, in this scenario, the update_deleted
conflict detection will not function correctly without
'track_commit_timestamp'.
```
IIUC, track_commit_timestamp is a GUC that designed mainly for conflict
detection, so it seems an unreasonable behavior to me if user enable this when
creating the sub but disable is afterwards. Besides, we documented that
update_deleted conflict would not be detected when track_commit_timestamp is
not enabled, so I am not sure if it's worth more effort adding checks for this
case.
2) With the new parameter name change to "retain_conflict_info", the error
message for both the 'CREATE SUBSCRIPTION' and 'ALTER SUBSCRIPTION'
commands needs to be updated accordingly.postgres=# create subscription sub11 connection 'dbname=postgres'
publication pub1 with (retain_conflict_info=on);
ERROR: detecting update_deleted conflicts requires
"track_commit_timestamp" to be enabled
postgres=# alter subscription sub12 set (retain_conflict_info=on);
ERROR: detecting update_deleted conflicts requires
"track_commit_timestamp" to be enabled- Change the message to something similar - "retaining conflict info requires
"track_commit_timestamp" to be enabled".
After thinking more, I changed this to a warning for now, because to detect
all necessary conflicts, user must enable the option anyway, and the same has
been documented for update/delete_origin_differs conflicts as well.
Best Regards,
Hou zj
On Wed, Jan 8, 2025 at 7:26 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thursday, January 9, 2025 9:48 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 3:00 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Wednesday, January 8, 2025 6:33 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond
<nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples
created by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will
remove dead tuples even when autovacuum is disabled.True, but why did it disable autovacuum? It seems that
case1-2_setup.sh doesn't specify fillfactor, which makes hot-updates lesslikely to happen.
IIUC, we disable autovacuum as a general practice in read-write tests
for stable TPS numbers.Okay. TBH I'm not sure what we can say with these results. At a glance, in a
typical bi-directional-like setup, we can interpret these results as that if
users turn retain_conflict_info on the TPS goes 50% down. But I'm not sure
this 50% dip is the worst case that users possibly face. It could be better in
practice thanks to autovacuum, or it also could go even worse due to further
bloats if we run the test longer.I think it shouldn't go worse because ideally the amount of bloat would not
increase beyond what we see here due to this patch unless there is some
misconfiguration that leads to one of the node not working properly (say it is
down). However, my colleague is running longer tests and we will share the
results soon.Suppose that users had 50% performance dip due to dead tuple retention for
update_deleted detection, is there any way for users to improve the situation?
For example, trying to advance slot.xmin more frequently might help to reduce
dead tuple accumulation. I think it would be good if we could have a way to
balance between the publisher performance and the subscriber performance.AFAICS, most of the time in each xid advancement is spent on waiting for the
target remote_lsn to be applied and flushed, so increasing the frequency could
not help. This can be proved to be reasonable in the testcase 4 shared by
Nisha[1], in that test, we do not request a remote_lsn but simply wait for the
commit_ts of incoming transaction to exceed the candidate_xid_time, the
regression is still the same.
True, but I think that not only more frequently asking the publisher
its status but also the apply worker frequently trying to advance the
RetainConflictInfoPhase and the launcher frequently trying to advance
the slot.xmin are important.
I think it indicates that we indeed need to wait
for this amount of time before applying all the transactions that have earlier
commit timestamp. IOW, the performance impact on the subscriber side is a
reasonable behavior if we want to detect the update_deleted conflict reliably.
It's reasonable behavior for this approach but it might not be a
reasonable outcome for users if they could be affected by such a
performance dip without no way to avoid it.
To closely look at what is happening in the apply worker and the
launcher, I did a quick test with the same setup, where running
pgbench with 30 clients to each of the publisher and subscriber (on
different pgbench tables so conflicts don't happen on the subscriber),
and I recorded how often the worker and the launcher tried to update
the worker's xmin and slot's xmin, respectively. During the 120
seconds test I observed that the apply worker advanced its
oldest_nonremovable_xid 10 times with 43 attempts and the launcher
advanced the slot's xmin 5 times with 20 attempts, which seems to be
less frequent. And there seems no way for users to increase these
frequencies. Actually, these XID advancements happened only early in
the test and in the later part there was almost no attempt to advance
XIDs (I described the reason below). Therefore, after 120 secs tests,
slot's xmin was 2366291 XIDs behind (TPS on the publisher and
subscriber were 15728 and 18052, respectively).
I think there 3 things we need to deal with:
1. The launcher could still be sleeping even after the worker updates
its oldest_nonremovable_xid. We compute the launcher's sleep time by
doubling the sleep time with 3min maximum time. When I started the
test, the launcher already entered 3min sleep, and it took a long time
to advance the slot.xmin for the first time. I think we can improve
this situation by having the worker send a signal to the launcher
after updating the worker's oldest_nonremovable_xid so that it can
quickly update the slot.xmin.
2. The apply worker doesn't advance RetainConflictInfoPhase from the
RCI_WAIT_FOR_LOCAL_FLUSH phase when it's busy. Regarding the phase
transition from RCI_WAIT_FOR_LOCAL_FLUSH to RCI_GET_CANDIDATE_XID, we
rely on calling maybe_advance_nonremovable_xid() (1) right after
transitioning to RCI_WAIT_FOR_LOCAL_FLUSH phase, (2) after receiving
'k' message, and (3) there is no available incoming data. If we miss
(1) opportunity (because we still need to wait for the local flush),
we effectively need to consume all available data to call
maybe_advance_nonremovable_xid() (note that the publisher doesn't need
to send 'k' (keepalive) message if it sends data frequently). In the
test, since I ran pgbench with 30 clients on the publisher and
therefore there were some apply delays, the apply worker took 25 min
to get out the inner apply loop in LogicalRepApplyLoop() and advance
its oldest_nonremovable_xid. I think we need to consider having more
opportunities to check the local flush LSN.
3. If the apply worker cannot catch up, it could enter to a bad loop;
the publisher sends huge amount of data -> the apply worker cannot
catch up -> it needs to wait for a longer time to advance its
oldest_nonremovable_xid -> more garbage are accumulated and then have
the apply more slow -> (looping). I'm not sure how to deal with this
point TBH. We might be able to avoid entering this bad loop once we
resolve the other two points.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = falseSince you disabled autovacuum on the subscriber, dead tuples created
by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?Test Run:
- Ran pgbench(read-write) on both the publisher and the subscriber with 30 clients for a duration of 120 seconds, collecting data over 5 runs.
- Note that pgbench was running for different tables on pub and sub.
(The scripts used for test "case1-2_measure.sh" and case1-2_setup.sh" are attached).Results:
Run# pub TPS sub TPS
1 32209 13704
2 32378 13684
3 32720 13680
4 31483 13681
5 31773 13813
median 32209 13684
regression 7% -53%What was the TPS on the subscriber when detect_update_deleted = false?
And how much were the tables bloated compared to when
detect_update_deleted = false?
Test results with 'retain_conflict_info=false', tested on v20 patches
where the parameter name is changed.
With 'retain_conflict_info' disabled, both the Publisher and
Subscriber sustain similar TPS, with no performance reduction observed
on either node.
Test Setup:
(used same setup as above test)
- Tests performed on pgHead+v20 patches
- Created a pub-sub replication setup.
- Parameters for both instances were:
autovacuum = false
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
Note: 'track_commit_timestamp' is disabled on Sub as not required for
retain_conflict_info=false.
Test Run:
- Pub and Sub had different pgbench tables with initial data of scale=100.
- Ran pgbench(read-write) on both the publisher and the subscriber
with 30 clients for a duration of 15 minutes, collecting data over 3
runs.
Results:
Run# pub TPS sub TPS
1 30533.29878 29161.33335
2 29931.30723 29520.89321
3 30665.54192 29440.92953
Median 30533.29878 29440.92953
pgHead median 30112.31203 28933.75013
regression 1% 2%
- Both Pub and Sub nodes have similar TPS in all runs, which is 1-2%
better than pgHead.
--
Thanks,
Nisha
On Friday, January 10, 2025 8:43 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 7:26 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Thursday, January 9, 2025 9:48 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 3:00 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:
On Wednesday, January 8, 2025 6:33 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
Hi,
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila
<amit.kapila16@gmail.com>
wrote:
On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond
<nisha.moond412@gmail.com> wrote:
[3] Test with pgbench run on both publisher and subscriber.
Test setup:
- Tests performed on pgHead + v16 patches
- Created a pub-sub replication system.
- Parameters for both instances were:
share_buffers = 30GB
min_wal_size = 10GB
max_wal_size = 20GB
autovacuum = false
Since you disabled autovacuum on the subscriber, dead tuples
created by non-hot updates are accumulated anyway regardless of
detect_update_deleted setting, is that right?I think hot-pruning mechanism during the update operation will
remove dead tuples even when autovacuum is disabled.True, but why did it disable autovacuum? It seems that
case1-2_setup.sh doesn't specify fillfactor, which makes hot-updatesless
likely to happen.
IIUC, we disable autovacuum as a general practice in read-write tests
for stable TPS numbers.Okay. TBH I'm not sure what we can say with these results. At a glance, in
a
typical bi-directional-like setup, we can interpret these results as that if
users turn retain_conflict_info on the TPS goes 50% down. But I'm notsure
this 50% dip is the worst case that users possibly face. It could be better in
practice thanks to autovacuum, or it also could go even worse due tofurther
bloats if we run the test longer.
Suppose that users had 50% performance dip due to dead tuple retentionfor
update_deleted detection, is there any way for users to improve the
situation?
For example, trying to advance slot.xmin more frequently might help to
reduce
dead tuple accumulation. I think it would be good if we could have a way to
balance between the publisher performance and the subscriberperformance.
AFAICS, most of the time in each xid advancement is spent on waiting for the
target remote_lsn to be applied and flushed, so increasing the frequencycould
not help. This can be proved to be reasonable in the testcase 4 shared by
Nisha[1], in that test, we do not request a remote_lsn but simply wait for the
commit_ts of incoming transaction to exceed the candidate_xid_time, the
regression is still the same.True, but I think that not only more frequently asking the publisher
its status but also the apply worker frequently trying to advance the
RetainConflictInfoPhase and the launcher frequently trying to advance
the slot.xmin are important.
I agree.
I think it indicates that we indeed need to wait
for this amount of time before applying all the transactions that have earlier
commit timestamp. IOW, the performance impact on the subscriber side is a
reasonable behavior if we want to detect the update_deleted conflict reliably.It's reasonable behavior for this approach but it might not be a
reasonable outcome for users if they could be affected by such a
performance dip without no way to avoid it.To closely look at what is happening in the apply worker and the
launcher, I did a quick test with the same setup, where running
pgbench with 30 clients to each of the publisher and subscriber (on
different pgbench tables so conflicts don't happen on the subscriber),
and I recorded how often the worker and the launcher tried to update
the worker's xmin and slot's xmin, respectively. During the 120
seconds test I observed that the apply worker advanced its
oldest_nonremovable_xid 10 times with 43 attempts and the launcher
advanced the slot's xmin 5 times with 20 attempts, which seems to be
less frequent. And there seems no way for users to increase these
frequencies. Actually, these XID advancements happened only early in
the test and in the later part there was almost no attempt to advance
XIDs (I described the reason below). Therefore, after 120 secs tests,
slot's xmin was 2366291 XIDs behind (TPS on the publisher and
subscriber were 15728 and 18052, respectively).
Thanks for testing ! It appears that the frequency observed in your tests is
higher than what we've experienced locally. Could you please share the scripts
you used and possibly the machine configuration? This information will help us
verify the differences in the data you've shared.
I think there 3 things we need to deal with:
Thanks for the suggestions. We will analyze them and share some top-up patches
for the suggested changes later.
Best Regards,
Hou zj
Dear Sawada-san,
Thanks for giving comments. I've created top-up patches for addressing them.
1. The launcher could still be sleeping even after the worker updates
its oldest_nonremovable_xid. We compute the launcher's sleep time by
doubling the sleep time with 3min maximum time. When I started the
test, the launcher already entered 3min sleep, and it took a long time
to advance the slot.xmin for the first time. I think we can improve
this situation by having the worker send a signal to the launcher
after updating the worker's oldest_nonremovable_xid so that it can
quickly update the slot.xmin.
Done in 0006. Worker sends a signal when its oldest_nonremovable_xid is updated.
Also, for the testing purpose, the maximum nap time is shortened to 10s when
retain_conflict_info is enabled. This value can be tuned based on results.
2. The apply worker doesn't advance RetainConflictInfoPhase from the
RCI_WAIT_FOR_LOCAL_FLUSH phase when it's busy. Regarding the phase
transition from RCI_WAIT_FOR_LOCAL_FLUSH to RCI_GET_CANDIDATE_XID,
we
rely on calling maybe_advance_nonremovable_xid() (1) right after
transitioning to RCI_WAIT_FOR_LOCAL_FLUSH phase, (2) after receiving
'k' message, and (3) there is no available incoming data. If we miss
(1) opportunity (because we still need to wait for the local flush),
we effectively need to consume all available data to call
maybe_advance_nonremovable_xid() (note that the publisher doesn't need
to send 'k' (keepalive) message if it sends data frequently). In the
test, since I ran pgbench with 30 clients on the publisher and
therefore there were some apply delays, the apply worker took 25 min
to get out the inner apply loop in LogicalRepApplyLoop() and advance
its oldest_nonremovable_xid. I think we need to consider having more
opportunities to check the local flush LSN.
Done in 0007. The worker now can advance its state machine even when it is busy.
Latest flush position is updating in wait_for_local_flush() too.
3. If the apply worker cannot catch up, it could enter to a bad loop;
the publisher sends huge amount of data -> the apply worker cannot
catch up -> it needs to wait for a longer time to advance its
oldest_nonremovable_xid -> more garbage are accumulated and then have
the apply more slow -> (looping). I'm not sure how to deal with this
point TBH. We might be able to avoid entering this bad loop once we
resolve the other two points.
I hope this issue is fixed because the worker can wait the local-flush even
while they are busy.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Attachments:
v21-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v21-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 4d44bb6da53b651608366cb1454dfe2378a43610 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v21 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 490 ++++++++++++++++++++-
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, 744 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172..49a4d3ab1b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1..0dcd4afe4c 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1182,7 +1182,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.
@@ -2356,7 +2360,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2380,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2393,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
@@ -2439,7 +2453,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 d331ab90d7..e3753d5d21 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a..31ebef831c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7af..31570377a9 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,60 @@ 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 */
+
+ /*
+ * 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 +394,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 +441,17 @@ 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 inline bool can_advance_nonremovable_xid(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,
@@ -3573,6 +3646,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
@@ -3651,6 +3725,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);
@@ -3677,6 +3753,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ /*
+ * Attempt to advance the non-removable transaction ID
+ * to avoid accumulating dead rows when the worker is
+ * busy.
+ */
+ if (can_advance_nonremovable_xid(&data))
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3776,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);
@@ -3706,6 +3814,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)
{
/*
@@ -3740,6 +3853,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,
@@ -3803,6 +3923,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
@@ -3838,7 +3960,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;
@@ -3916,6 +4037,373 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * 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;
+
+ /*
+ * Return to wait for the changes to be applied.
+ *
+ * XXX The remote flush location (last_flushpos) is updated only when
+ * feedback is sent to the server. So, the advancement of
+ * oldest_nonremovable_xid may be delayed. We can always update
+ * last_flushpos here if we notice such a delay.
+ */
+ 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;
+
+ /* process the next phase */
+ maybe_advance_nonremovable_xid(data, false);
+}
+
+/*
+ * Determine if we can attempt to advance transaction ID.
+ */
+static inline bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
+{
+ return data->phase == RCI_GET_CANDIDATE_XID;
+}
+
+/*
+ * 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 a0782b1bbf..1299ed1eb5 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f88..064841b482 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 30b2775952..b09486017f 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 20777f7d5a..d2452904dd 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe..bd8b17a6d0 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 eb93debe10..bac9cc412b 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2475,6 +2475,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.43.0
v21-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v21-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From a3fc09aa1450078a08e8f1ae6e0c697ac9e8aac6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v21 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 | 224 ++++++++++++++++++
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/slot.c | 34 ++-
src/include/replication/slot.h | 11 +-
10 files changed, 287 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index f1ab614575..e613e4ced1 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4879,6 +4879,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581a..feb026d0c5 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b..e20b48a49a 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1..e95e0cd271 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea20..d49ba78b13 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e..3cf1e05539 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 31ebef831c..350c86540f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,12 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -100,6 +106,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1119,6 +1129,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1160,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();
@@ -1167,14 +1182,49 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
+
+ /*
+ * 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 the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1257,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1311,153 @@ 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);
+ return;
+ }
+
+ /*
+ * Initially create persistent slot as ephemeral - that allows us to
+ * nicely handle errors during initialization because it'll get dropped if
+ * this transaction fails. We'll make it persistent after safely
+ * initializing the xmin of the slot.
+ */
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf..64b43346a9 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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/slot.c b/src/backend/replication/slot.c
index b30e0473e1..3a3884b382 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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/slot.h b/src/include/replication/slot.h
index bf62b36ad0..ea0485efa2 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.
*
@@ -261,7 +268,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);
--
2.43.0
v21-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v21-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 72b8706f41244de084c05cb9dc52333ae95a9db5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 9 Jan 2025 16:44:53 +0800
Subject: [PATCH v21 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 | 46 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 438 insertions(+), 124 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cc6cf9bef0..2835a1e6cc 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8041,6 +8041,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a08..bde698348f 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2160,7 +2160,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007..e5415c3150 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd271..61374b74d1 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413..39cfae43d6 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 7a595c84db..cee60482b6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1360,7 +1360,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 3cf1e05539..51bd52e48a 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();
{
@@ -2193,6 +2267,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 350c86540f..b3b57daf19 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -158,6 +158,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);
@@ -1161,6 +1162,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();
@@ -1181,12 +1183,21 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * 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.
+ */
+ if (sub->retainconflictinfo)
{
- can_advance_xmin = false;
- continue;
+ retain_conflict_info = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1195,10 +1206,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;
@@ -1224,7 +1236,8 @@ ApplyLauncherMain(Datum main_arg)
* 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 (retain_conflict_info)
+ create_conflict_slot_if_not_exists();
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1262,7 +1275,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1276,12 +1289,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1312,7 +1327,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
@@ -1358,7 +1373,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1369,7 +1384,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1446,8 +1464,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 31570377a9..5dcd01e686 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4102,6 +4102,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
@@ -4503,6 +4507,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 ||
@@ -4512,7 +4525,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 8f73a5df95..7940dc5803 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1..0be12d05c9 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc..70489b9374 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b60..1f84ed292a 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f..58395ab31c 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index d5543fd62b..92e0358e39 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6644,7 +6644,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)
{
@@ -6712,6 +6712,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 81cbf10aa2..e0be3c6fd6 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 6194b757d5..1d1e0dffb9 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 1443e1d929..bff4cc051d 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 007c9e7037..c65397e5ac 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;
--
2.43.0
v21-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v21-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From d5b3e6b91a70ffa3b03ed8e9af3b68e2340aefb8 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v21 4/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 156 ++++++++++++++++++
2 files changed, 157 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f..8bf4a83ea6 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 0000000000..7a69c13486
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,156 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.43.0
v21-0005-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v21-0005-Support-the-conflict-detection-for-update_delete.patchDownload
From 54a22318b70bd3f083b9558e4748455ee5ae6648 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v21 5/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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index bde698348f..7e8c959a2d 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d0d176cc54..878e955e62 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index cee60482b6..456e6835db 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,6 +1371,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 e3e4e41ac3..f08e727605 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88..0eedbd2b16 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 5dcd01e686..bea9895363 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2742,6 +2742,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2758,15 +2761,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2777,7 +2776,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2796,19 +2795,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3129,9 +3137,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3140,20 +3148,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3162,7 +3179,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3173,7 +3190,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 5f8d20a406..b289f5d45c 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2087,7 +2087,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2109,17 +2109,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index b37e8a6f88..e6c2f1d342 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 f8a8d03e53..83ccfd1b50 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"
@@ -670,7 +671,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 37454dc951..91224b6ea6 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 3014d047fe..2f5d4c970c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,6 +2144,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,
@@ -2151,7 +2152,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 7a69c13486..65e4033cb5 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -113,6 +118,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -120,6 +132,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.43.0
v21-0006-Make-launcher-wake-up-more-frequently.patchapplication/octet-stream; name=v21-0006-Make-launcher-wake-up-more-frequently.patchDownload
From ed5990361c5266744ca888403b5001526bfe1ed7 Mon Sep 17 00:00:00 2001
From: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Date: Fri, 10 Jan 2025 10:27:14 +0000
Subject: [PATCH v21 6/7] Make launcher wake up more frequently
---
src/backend/replication/logical/launcher.c | 22 ++++++++++++++++++----
src/backend/replication/logical/worker.c | 3 +++
src/include/replication/logicallauncher.h | 1 +
3 files changed, 22 insertions(+), 4 deletions(-)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index b3b57daf19..5c4ab223c7 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -52,6 +52,13 @@
*/
#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+/*
+ * Max sleep time (10s) between cycles update the xmin; If one of the
+ * subscriptions has retain_conflict_info = true, this will be the max sleep
+ * time of the cycle.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 10000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -97,7 +104,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);
@@ -1117,7 +1123,14 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Request wakeup of the launcher immediately.
+ *
+ * Apart from ApplyLauncherWakeupAtCommit(), this does not wait committing
+ * current transactions. Should be used when oldest_nonremovable_xid of apply
+ * workers is updated.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1440,19 +1453,20 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
* Update the sleep time before the next slot update.
*
* If there is no slot activity, the wait time between sync cycles will double
- * (up to a maximum of 3 minutes). If there is some slot activity, the wait
+ * (up to a maximum of 10 seconds). If there is some slot activity, the wait
* time between sync cycles is reset to the minimum (200ms).
*/
static void
compute_slot_update_naptime(bool slot_updated, long *sleep_time)
{
+
if (!slot_updated)
{
/*
* The slot was not updated, so double the sleep time, but not beyond
* the maximum allowable value.
*/
- *sleep_time = Min(*sleep_time * 2, DEFAULT_NAPTIME_PER_CYCLE);
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
}
else
{
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index bea9895363..5fa382ab3e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4358,6 +4358,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/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f330..7b29f1814d 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);
--
2.43.0
v21-0007-Update-flush-location-more-frequently.patchapplication/octet-stream; name=v21-0007-Update-flush-location-more-frequently.patchDownload
From aea4f9222eb53cdb9c49190d79bfb6659c44f536 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 10 Jan 2025 09:58:15 +0800
Subject: [PATCH v21 7/7] Update flush location more frequently
---
src/backend/replication/logical/worker.c | 28 ++++++++++--------------
1 file changed, 11 insertions(+), 17 deletions(-)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5fa382ab3e..453bd22432 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -448,7 +448,6 @@ 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 inline bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
static void adjust_xid_advance_interval(RetainConflictInfoData *data,
bool new_xid_found);
@@ -3771,13 +3770,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
apply_dispatch(&s);
- /*
- * Attempt to advance the non-removable transaction ID
- * to avoid accumulating dead rows when the worker is
- * busy.
- */
- if (can_advance_nonremovable_xid(&data))
- maybe_advance_nonremovable_xid(&data, false);
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -4300,6 +4293,10 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ XLogRecPtr writepos;
+ XLogRecPtr flushpos;
+ bool have_pending_txes;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4334,6 +4331,12 @@ wait_for_local_flush(RetainConflictInfoData *data)
if (!AllTablesyncsReady())
return;
+ /* Fetch and update latest remote flush position */
+ get_flush_position(&writepos, &flushpos, &have_pending_txes);
+
+ if (flushpos > last_flushpos)
+ last_flushpos = flushpos;
+
/*
* Return to wait for the changes to be applied.
*
@@ -4378,15 +4381,6 @@ wait_for_local_flush(RetainConflictInfoData *data)
maybe_advance_nonremovable_xid(data, false);
}
-/*
- * Determine if we can attempt to advance transaction ID.
- */
-static inline bool
-can_advance_nonremovable_xid(RetainConflictInfoData *data)
-{
- return data->phase == RCI_GET_CANDIDATE_XID;
-}
-
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.43.0
On Fri, Jan 10, 2025 at 6:13 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
3. If the apply worker cannot catch up, it could enter to a bad loop;
the publisher sends huge amount of data -> the apply worker cannot
catch up -> it needs to wait for a longer time to advance its
oldest_nonremovable_xid -> more garbage are accumulated and then have
the apply more slow -> (looping). I'm not sure how to deal with this
point TBH. We might be able to avoid entering this bad loop once we
resolve the other two points.
I don't think we can avoid accumulating garbage especially when the
workload on the publisher is more. Consider the current case being
discussed, on the publisher, we have 30 clients performing read-write
operations and there is only one pair of reader (walsender) and writer
(apply_worker) to perform all those write operations on the
subscriber. It can never match the speed and the subscriber side is
bound to have less performance (or accumulate more bloat) irrespective
of its workload. If there is one client on the publisher performing
operation, we won't see much degradation but as the number of clients
increases, the performance degradation (and bloat) will keep on
increasing.
There are other scenarios that can lead to the same situation, such as
a large table sync, the subscriber node being down for sometime, etc.
Basically, any case where apply_side lags by a large amount from the
remote node.
One idea to prevent the performance degradation or bloat increase is
to invalidate the slot, once we notice that subscriber lags (in terms
of WAL apply) behind the publisher by a certain threshold. Say we have
max_lag (or max_lag_behind_remote) (defined in terms of seconds)
subscription option which allows us to stop calculating
oldest_nonremovable_xid for that subscription. We can indicate that
via some worker_level parameter. Once all the subscriptions on a node
that has enabled retain_conflict_info have stopped calculating
oldest_nonremovable_xid, we can invalidate the slot. Now, users can
check this and need to disable/enable retain_conflict_info to again
start retaining the required information. The other way could be that
instead of invalidating the slot, we directly drop/re-create the slot
or increase its xmin. If we choose to advance the slot automatically
without user intervention, we need to let users know via LOG and or
via information in the view.
I think such a mechanism via the new option max_lag will address your
concern: "It's reasonable behavior for this approach but it might not
be a reasonable outcome for users if they could be affected by such a
performance dip without no way to avoid it." as it will provide a way
to avoid performance dip only when there is a possibility of such a
dip.
I mentioned max_lag as a subscription option instead of a GUC because
it applies only to subscriptions that have enabled
retain_conflict_info but we can consider it to be a GUC if you and
others think so provided the above proposal sounds reasonable. Also,
max_lag could be defined in terms of LSN as well but I think time
would be easy to configure.
Thoughts?
--
With Regards,
Amit Kapila.
Here are the performance test results and analysis with the recent patches.
Test Setup:
- Created Pub and Sub nodes with logical replication and below configurations.
autovacuum_naptime = '30s'
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
track_commit_timestamp = on (only on Sub node).
- Pub and Sub had different pgbench tables with initial data of scale=100.
-------------------------------
Case-0: Collected data on pgHead
-------------------------------
- Ran pgbench(read-write) on both the publisher and the subscriber
with 30 clients for a duration of 15 minutes, collecting data over 3
runs.
Results:
Run# pub_TPS sub_TPS
1 30551.63471 29476.81709
2 30112.31203 28933.75013
3 29599.40383 28379.4977
Median 30112.31203 28933.75013
-------------------------------
Case-1: Long run(15-minutes) tests when retain_conflitc_info=ON
-------------------------------
- Code: pgHead + v19 patches.
- At Sub set autovacuum=false.
- Ran pgbench(read-write) on both the publisher and the subscriber
with 30 clients for a duration of 15 minutes, collecting data over 3
runs.
Results:
Run# pub_TPS sub_TPS
1 30326.57637 4890.410972
2 30412.85115 4787.192754
3 30860.13879 4864.549117
Median 30412.85115 4864.549117
regression 1% -83%
- A 15-minute pgbench run test showed higher reduction in the sub's
TPS. As the test run time increased the TPS reduced further at the Sub
node.
-------------------------------
Case-2 : Re-ran the case-1 with autovacuum enabled and running every 30 seconds.
-------------------------------
- Code: pgHead + v19 patches.
- At Sub set autovacuum=true.
- Also measured the frequency of slot.xmin and the worker's
oldest_nonremovable_xid updates.
Results:
Run# pub_TPS sub_TPS #slot.xmin_updates
#worker's_oldest_nonremovable_xid_updates
1 31080.30944 4573.547293 0 1
regression 3% -84%
- Autovacuum did not help in improving the Sub's TPS.
- The slot's xmin was not advanced.
~~~~
Observations and RCA for TPS reduction in above tests:
- The launcher was not able to advance slot.xmin during the 15-minute
pgbench run, leading to increased dead tuple accumulation on the
subscriber node.
- The launcher failed to advance slot.xmin because the apply worker
could not set the oldest_nonremovable_xid early and frequently enough
due to following two reasons -
1) For large pgbench tables (scale=100), the tablesync takes time
to complete, forcing the apply worker to wait before updating its
oldest_nonremovable_xid.
2) With 30 clients generating operations at a pace that a single
apply worker cannot match, the worker fails to catch up with the
rapidly increasing remote_lsn, lagging behind the Publisher's LSN
throughout the 15-minute run.
Considering the above reasons, for better performance measurements,
collected data when table_sync is off, with a varying number of
clients on the publisher node. Below test used the v21 patch set,
which also includes improvement patches (006 and 007) for more
frequent slot.xmin updates.
-------------------------------
Case-3: Create the subscription with option "copy_data=false", so, no
tablesync in the picture.
-------------------------------
Test setup:
- Code: pgHead + v21 patches.
- Created Pub and Sub nodes with logical replication and below configurations.
autovacuum_naptime = '30s'
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
track_commit_timestamp = on (only on Sub node).
- The Pub and Sub had different pgbench tables with initial data of scale=100.
- Ran pgbench(read-write) on both the pub and the sub for a duration
of 15 minutes, using 30 clients on the Subscriber while varying the
number of clients on the Publisher.
- In addition to TPS, the frequency of slot.xmin and the worker's
oldest_nonremovable_xid updates was also measured.
Observations:
- As the number of clients on the publisher increased, the
publisher's TPS improved, but the subscriber's TPS dropped
significantly.
- The frequency of slot.xmin updates also declined with more clients
on the publisher, indicating that the apply worker updated its
oldest_nonremovable_xid less frequently as the read-write operations
on the publisher increased.
Results:
#Pub-clients pubTPS pubTPS_increament subTPS pubTPS_reduction
#slot.xmin_updates #worker's_oldest_nonremovable_xid_updates
1 1364.487898 0 35000.06738 0 6976 6977
2 2706.100445 98% 32297.81408 -8% 5838 5839
4 5079.522778 272% 8581.034791 -75% 268 269
30 31308.18524 2195% 5324.328696 -85% 4 5
Note: In the above result table, the column -
- "PubTPS_increment" represents the % improvement in the Pub's
TPS compared to its TPS in the initial run with #Pub-clients=1 and
- "SubTPS_reduction" indicates the % decrease in the Sub's TPS
compared to its TPS in the initial run with #Pub-clients=1.
~~~~
Conclusion:
There is some improvement in slot.xmin update frequency with
table_sync off and the additional patches that updates slot's xmin
aggressively.
However, the key point is that with a large number of clients
generating write operations, apply worker LAGs with a large margin
leading to non-updation of slot's xmin as the test run time increases.
This is also visible [in case-3] that with only 1 client on publisher,
there is no degradation on the subscriber. As the number of clients
increases, the degradation also increases.
Based on this test analysis I can say that we need some way/option to
invalidate such slots that LAG by a threshold margin, as mentioned at
[1]: /messages/by-id/CAA4eK1Jyo4odkVsnSeAWPh8Wgpw12EbS9q8s_eN14LtcFNXCSA@mail.gmail.com
~~~~
(Attached the test scripts used for above tests)
[1]: /messages/by-id/CAA4eK1Jyo4odkVsnSeAWPh8Wgpw12EbS9q8s_eN14LtcFNXCSA@mail.gmail.com
--
Thanks,
Nisha
On Sun, Jan 12, 2025 at 10:36 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jan 10, 2025 at 6:13 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
3. If the apply worker cannot catch up, it could enter to a bad loop;
the publisher sends huge amount of data -> the apply worker cannot
catch up -> it needs to wait for a longer time to advance its
oldest_nonremovable_xid -> more garbage are accumulated and then have
the apply more slow -> (looping). I'm not sure how to deal with this
point TBH. We might be able to avoid entering this bad loop once we
resolve the other two points.I don't think we can avoid accumulating garbage especially when the
workload on the publisher is more. Consider the current case being
discussed, on the publisher, we have 30 clients performing read-write
operations and there is only one pair of reader (walsender) and writer
(apply_worker) to perform all those write operations on the
subscriber. It can never match the speed and the subscriber side is
bound to have less performance (or accumulate more bloat) irrespective
of its workload. If there is one client on the publisher performing
operation, we won't see much degradation but as the number of clients
increases, the performance degradation (and bloat) will keep on
increasing.There are other scenarios that can lead to the same situation, such as
a large table sync, the subscriber node being down for sometime, etc.
Basically, any case where apply_side lags by a large amount from the
remote node.One idea to prevent the performance degradation or bloat increase is
to invalidate the slot, once we notice that subscriber lags (in terms
of WAL apply) behind the publisher by a certain threshold. Say we have
max_lag (or max_lag_behind_remote) (defined in terms of seconds)
subscription option which allows us to stop calculating
oldest_nonremovable_xid for that subscription. We can indicate that
via some worker_level parameter. Once all the subscriptions on a node
that has enabled retain_conflict_info have stopped calculating
oldest_nonremovable_xid, we can invalidate the slot. Now, users can
check this and need to disable/enable retain_conflict_info to again
start retaining the required information. The other way could be that
instead of invalidating the slot, we directly drop/re-create the slot
or increase its xmin. If we choose to advance the slot automatically
without user intervention, we need to let users know via LOG and or
via information in the view.I think such a mechanism via the new option max_lag will address your
concern: "It's reasonable behavior for this approach but it might not
be a reasonable outcome for users if they could be affected by such a
performance dip without no way to avoid it." as it will provide a way
to avoid performance dip only when there is a possibility of such a
dip.I mentioned max_lag as a subscription option instead of a GUC because
it applies only to subscriptions that have enabled
retain_conflict_info but we can consider it to be a GUC if you and
others think so provided the above proposal sounds reasonable. Also,
max_lag could be defined in terms of LSN as well but I think time
would be easy to configure.Thoughts?
I agree that we cannot avoid accumulating dead tuples when the
workload on the publisher is more, and which affects the subscriber
performance. What we need to do is to update slot's xmin as quickly as
possible to minimize the dead tuple accumulation at least when the
subscriber is not much behind. If there is a tradeoff for doing so
(e.g., vs. the publisher performance), we need to provide a way for
users to balance it. The max_lag idea sounds interesting for the case
where the subscriber is much behind. Probably we can visit this idea
as a new feature after completing this feature.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tue, Jan 14, 2025 at 7:14 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Sun, Jan 12, 2025 at 10:36 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I don't think we can avoid accumulating garbage especially when the
workload on the publisher is more. Consider the current case being
discussed, on the publisher, we have 30 clients performing read-write
operations and there is only one pair of reader (walsender) and writer
(apply_worker) to perform all those write operations on the
subscriber. It can never match the speed and the subscriber side is
bound to have less performance (or accumulate more bloat) irrespective
of its workload. If there is one client on the publisher performing
operation, we won't see much degradation but as the number of clients
increases, the performance degradation (and bloat) will keep on
increasing.There are other scenarios that can lead to the same situation, such as
a large table sync, the subscriber node being down for sometime, etc.
Basically, any case where apply_side lags by a large amount from the
remote node.One idea to prevent the performance degradation or bloat increase is
to invalidate the slot, once we notice that subscriber lags (in terms
of WAL apply) behind the publisher by a certain threshold. Say we have
max_lag (or max_lag_behind_remote) (defined in terms of seconds)
subscription option which allows us to stop calculating
oldest_nonremovable_xid for that subscription. We can indicate that
via some worker_level parameter. Once all the subscriptions on a node
that has enabled retain_conflict_info have stopped calculating
oldest_nonremovable_xid, we can invalidate the slot. Now, users can
check this and need to disable/enable retain_conflict_info to again
start retaining the required information. The other way could be that
instead of invalidating the slot, we directly drop/re-create the slot
or increase its xmin. If we choose to advance the slot automatically
without user intervention, we need to let users know via LOG and or
via information in the view.I think such a mechanism via the new option max_lag will address your
concern: "It's reasonable behavior for this approach but it might not
be a reasonable outcome for users if they could be affected by such a
performance dip without no way to avoid it." as it will provide a way
to avoid performance dip only when there is a possibility of such a
dip.I mentioned max_lag as a subscription option instead of a GUC because
it applies only to subscriptions that have enabled
retain_conflict_info but we can consider it to be a GUC if you and
others think so provided the above proposal sounds reasonable. Also,
max_lag could be defined in terms of LSN as well but I think time
would be easy to configure.Thoughts?
I agree that we cannot avoid accumulating dead tuples when the
workload on the publisher is more, and which affects the subscriber
performance. What we need to do is to update slot's xmin as quickly as
possible to minimize the dead tuple accumulation at least when the
subscriber is not much behind. If there is a tradeoff for doing so
(e.g., vs. the publisher performance), we need to provide a way for
users to balance it.
As of now, I can't think of a way to throttle the publisher when the
apply_worker lags. Basically, we need some way to throttle (reduce the
speed of backends) when the apply worker is lagging behind a threshold
margin. Can you think of some way? I thought if one notices frequent
invalidation of the launcher's slot due to max_lag, then they can
rebalance their workload on the publisher.
The max_lag idea sounds interesting for the case
where the subscriber is much behind. Probably we can visit this idea
as a new feature after completing this feature.
Sure, but what will be our answer to users for cases where the
performance tanks due to bloat accumulation? The tests show that once
the apply_lag becomes large, it becomes almost impossible for the
apply worker to catch up (or take a very long time) and advance the
slot's xmin. The users can disable retain_conflict_info to bring back
the performance and get rid of bloat but I thought it would be easier
for users to do that if we have some knob where they don't need to
wait till actually the problem of bloat/performance dip happens.
--
With Regards,
Amit Kapila.
On Mon, Jan 13, 2025 at 8:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jan 14, 2025 at 7:14 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Sun, Jan 12, 2025 at 10:36 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I don't think we can avoid accumulating garbage especially when the
workload on the publisher is more. Consider the current case being
discussed, on the publisher, we have 30 clients performing read-write
operations and there is only one pair of reader (walsender) and writer
(apply_worker) to perform all those write operations on the
subscriber. It can never match the speed and the subscriber side is
bound to have less performance (or accumulate more bloat) irrespective
of its workload. If there is one client on the publisher performing
operation, we won't see much degradation but as the number of clients
increases, the performance degradation (and bloat) will keep on
increasing.There are other scenarios that can lead to the same situation, such as
a large table sync, the subscriber node being down for sometime, etc.
Basically, any case where apply_side lags by a large amount from the
remote node.One idea to prevent the performance degradation or bloat increase is
to invalidate the slot, once we notice that subscriber lags (in terms
of WAL apply) behind the publisher by a certain threshold. Say we have
max_lag (or max_lag_behind_remote) (defined in terms of seconds)
subscription option which allows us to stop calculating
oldest_nonremovable_xid for that subscription. We can indicate that
via some worker_level parameter. Once all the subscriptions on a node
that has enabled retain_conflict_info have stopped calculating
oldest_nonremovable_xid, we can invalidate the slot. Now, users can
check this and need to disable/enable retain_conflict_info to again
start retaining the required information. The other way could be that
instead of invalidating the slot, we directly drop/re-create the slot
or increase its xmin. If we choose to advance the slot automatically
without user intervention, we need to let users know via LOG and or
via information in the view.I think such a mechanism via the new option max_lag will address your
concern: "It's reasonable behavior for this approach but it might not
be a reasonable outcome for users if they could be affected by such a
performance dip without no way to avoid it." as it will provide a way
to avoid performance dip only when there is a possibility of such a
dip.I mentioned max_lag as a subscription option instead of a GUC because
it applies only to subscriptions that have enabled
retain_conflict_info but we can consider it to be a GUC if you and
others think so provided the above proposal sounds reasonable. Also,
max_lag could be defined in terms of LSN as well but I think time
would be easy to configure.Thoughts?
I agree that we cannot avoid accumulating dead tuples when the
workload on the publisher is more, and which affects the subscriber
performance. What we need to do is to update slot's xmin as quickly as
possible to minimize the dead tuple accumulation at least when the
subscriber is not much behind. If there is a tradeoff for doing so
(e.g., vs. the publisher performance), we need to provide a way for
users to balance it.As of now, I can't think of a way to throttle the publisher when the
apply_worker lags. Basically, we need some way to throttle (reduce the
speed of backends) when the apply worker is lagging behind a threshold
margin. Can you think of some way? I thought if one notices frequent
invalidation of the launcher's slot due to max_lag, then they can
rebalance their workload on the publisher.
I don't have any ideas other than invalidating the launcher's slot
when the apply lag is huge. We can think of invalidating the
launcher's slot for some reasons such as the replay lag, the age of
slot's xmin, and the duration.
The max_lag idea sounds interesting for the case
where the subscriber is much behind. Probably we can visit this idea
as a new feature after completing this feature.Sure, but what will be our answer to users for cases where the
performance tanks due to bloat accumulation? The tests show that once
the apply_lag becomes large, it becomes almost impossible for the
apply worker to catch up (or take a very long time) and advance the
slot's xmin. The users can disable retain_conflict_info to bring back
the performance and get rid of bloat but I thought it would be easier
for users to do that if we have some knob where they don't need to
wait till actually the problem of bloat/performance dip happens.
Probably retaining dead tuples based on the time duration or its age
might be other solutions, it would increase a risk of not being able
to detect update_deleted conflict though. I think in any way as long
as we accumulate dead tulpes to detect update_deleted conflicts, it
would be a tradeoff between reliably detecting update_deleted
conflicts and the performance.
As for detecting update_deleted conflicts, we probably don't need the
whole tuple data of deleted tuples. It would be sufficient if we can
check XIDs of deleted tuple to get their origins and commit
timestamps. Probably the same is true for the old version of updated
tuple in terms of detecting update_origin_differ conflicts. If my
understanding is right, probably we can remove only the tuple data of
dead tuples that are older than a xmin horizon (excluding the
launcher's xmin), while leaving the heap tuple header, which can
minimize the table bloat.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wed, Jan 15, 2025 at 5:57 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Jan 13, 2025 at 8:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
As of now, I can't think of a way to throttle the publisher when the
apply_worker lags. Basically, we need some way to throttle (reduce the
speed of backends) when the apply worker is lagging behind a threshold
margin. Can you think of some way? I thought if one notices frequent
invalidation of the launcher's slot due to max_lag, then they can
rebalance their workload on the publisher.I don't have any ideas other than invalidating the launcher's slot
when the apply lag is huge. We can think of invalidating the
launcher's slot for some reasons such as the replay lag, the age of
slot's xmin, and the duration.
Right, this is exactly where we are heading. I think we can add
reasons step-wise. For example, as a first step, we can invalidate the
slot due to replay LAG. Then, slowly, we can add other reasons as
well.
One thing that needs more discussion is the exact way to invalidate a
slot. I have mentioned a couple of ideas in my previous email which I
am writing again: "If we just invalidate the slot, users can check the
status of the slot and need to disable/enable retain_conflict_info
again to start retaining the required information. This would be
required because we can't allow system slots (slots created
internally) to be created by users. The other way could be that
instead of invalidating the slot, we directly drop/re-create the slot
or increase its xmin. If we choose to advance the slot automatically
without user intervention, we need to let users know via LOG and or
via information in the view."
The max_lag idea sounds interesting for the case
where the subscriber is much behind. Probably we can visit this idea
as a new feature after completing this feature.Sure, but what will be our answer to users for cases where the
performance tanks due to bloat accumulation? The tests show that once
the apply_lag becomes large, it becomes almost impossible for the
apply worker to catch up (or take a very long time) and advance the
slot's xmin. The users can disable retain_conflict_info to bring back
the performance and get rid of bloat but I thought it would be easier
for users to do that if we have some knob where they don't need to
wait till actually the problem of bloat/performance dip happens.Probably retaining dead tuples based on the time duration or its age
might be other solutions, it would increase a risk of not being able
to detect update_deleted conflict though. I think in any way as long
as we accumulate dead tulpes to detect update_deleted conflicts, it
would be a tradeoff between reliably detecting update_deleted
conflicts and the performance.
Right, and users have an option for it. Say, if they set max_lag as -1
(or some special value), we won't invalidate the slot, so the
update_delete conflict can be detected with complete reliability. At
this stage, it is okay if this information is LOGGED and displayed via
a system view. We need more thoughts while working on the CONFLICT
RESOLUTION patch such as we may need to additionally display a WARNING
or ERROR if the remote_tuples commit_time is earlier than the last
time slot is invalidated. I don't want to go in a detailed discussion
at this point but just wanted you to know that we will need additional
work for the resolution of update_delete conflicts to avoid
inconsistency.
As for detecting update_deleted conflicts, we probably don't need the
whole tuple data of deleted tuples. It would be sufficient if we can
check XIDs of deleted tuple to get their origins and commit
timestamps. Probably the same is true for the old version of updated
tuple in terms of detecting update_origin_differ conflicts. If my
understanding is right, probably we can remove only the tuple data of
dead tuples that are older than a xmin horizon (excluding the
launcher's xmin), while leaving the heap tuple header, which can
minimize the table bloat.
I am afraid that is not possible because even to detect the conflict,
we first need to find the matching tuple on the subscriber node. If
the replica_indentity or primary_key is present in the table, we try
to save that and transaction info but that won't be simple either.
Also, if RI or primary_key is not there, we need an entire tuple to
match. We need a concept of tombstone tables (or we can call it a
dead-rows-store) where old data is stored reliably till we don't need
it. We have discussed briefly that idea previously [1]/messages/by-id/CAJpy0uCov4JfZJeOvY0O21_gk9bcgNUDp4jf8+BbMp+EAv8cVQ@mail.gmail.com[2]/messages/by-id/e4cdb849-d647-4acf-aabe-7049ae170fbf@enterprisedb.com and decided
to move forward with an idea to retain the dead tuples idea based on
the theory that we already use similar ideas at other places.
BTW, a related point to note is that we need to retain the
conflict_info even to detect origin_differ conflict with complete
reliability. We need only commit_ts information for that case. See
analysis [3]/messages/by-id/OSCPR01MB14966F6B816880165E387758AF5112@OSCPR01MB14966.jpnprd01.prod.outlook.com.
[1]: /messages/by-id/CAJpy0uCov4JfZJeOvY0O21_gk9bcgNUDp4jf8+BbMp+EAv8cVQ@mail.gmail.com
[2]: /messages/by-id/e4cdb849-d647-4acf-aabe-7049ae170fbf@enterprisedb.com
[3]: /messages/by-id/OSCPR01MB14966F6B816880165E387758AF5112@OSCPR01MB14966.jpnprd01.prod.outlook.com
--
With Regards,
Amit Kapila.
On Wednesday, January 15, 2025 12:08 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Hi,
On Wed, Jan 15, 2025 at 5:57 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:On Mon, Jan 13, 2025 at 8:39 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
As of now, I can't think of a way to throttle the publisher when the
apply_worker lags. Basically, we need some way to throttle (reduce
the speed of backends) when the apply worker is lagging behind a
threshold margin. Can you think of some way? I thought if one
notices frequent invalidation of the launcher's slot due to max_lag,
then they can rebalance their workload on the publisher.I don't have any ideas other than invalidating the launcher's slot
when the apply lag is huge. We can think of invalidating the
launcher's slot for some reasons such as the replay lag, the age of
slot's xmin, and the duration.Right, this is exactly where we are heading. I think we can add reasons
step-wise. For example, as a first step, we can invalidate the slot due to replay
LAG. Then, slowly, we can add other reasons as well.One thing that needs more discussion is the exact way to invalidate a slot. I
have mentioned a couple of ideas in my previous email which I am writing
again: "If we just invalidate the slot, users can check the status of the slot and
need to disable/enable retain_conflict_info again to start retaining the required
information. This would be required because we can't allow system slots (slots
created
internally) to be created by users. The other way could be that instead of
invalidating the slot, we directly drop/re-create the slot or increase its xmin. If
we choose to advance the slot automatically without user intervention, we need
to let users know via LOG and or via information in the view."
In the latest version, we implemented a simpler approach that allows the apply
worker to directly advance the oldest_nonremovable_xid if the waiting time
exceeds the newly introduced option's limit. I've named this option
"max_conflict_retention_duration," as it aligns better with the conflict
detection concept and the "retain_conflict_info" option.
During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker evaluates
how much time it has spent waiting. If this duration exceeds the
max_conflict_retention_duration, the worker directly advances the
oldest_nonremovable_xid and logs a message indicating the forced advancement of
the non-removable transaction ID.
This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler because we
can avoid adding 1) new slot invalidation type due to apply lag, 2) new field
lag_behind in shared memory (MyLogicalRepWorker) to indicate when the lag
exceeds the limit, and 3) additional logic in the launcher to handle each
worker's lag status.
In the slot invalidation, user would be able to confirm if the current by
checking if the slot in pg_replication_slot in invalidated or not, while in the
simpler approach mentioned, user could only confirm that by checking the LOGs.
What do you think ? If we prefer the slot invalidation approach, I can do that
part in next version.
The max_lag idea sounds interesting for the case
where the subscriber is much behind. Probably we can visit this
idea as a new feature after completing this feature.Sure, but what will be our answer to users for cases where the
performance tanks due to bloat accumulation? The tests show that
once the apply_lag becomes large, it becomes almost impossible for
the apply worker to catch up (or take a very long time) and advance
the slot's xmin. The users can disable retain_conflict_info to bring
back the performance and get rid of bloat but I thought it would be
easier for users to do that if we have some knob where they don't
need to wait till actually the problem of bloat/performance dip happens.Probably retaining dead tuples based on the time duration or its age
might be other solutions, it would increase a risk of not being able
to detect update_deleted conflict though. I think in any way as long
as we accumulate dead tulpes to detect update_deleted conflicts, it
would be a tradeoff between reliably detecting update_deleted
conflicts and the performance.Right, and users have an option for it. Say, if they set max_lag as -1 (or some
special value), we won't invalidate the slot, so the update_delete conflict can
be detected with complete reliability. At this stage, it is okay if this information
is LOGGED and displayed via a system view. We need more thoughts while
working on the CONFLICT RESOLUTION patch such as we may need to
additionally display a WARNING or ERROR if the remote_tuples commit_time is
earlier than the last time slot is invalidated. I don't want to go in a detailed
discussion at this point but just wanted you to know that we will need
additional work for the resolution of update_delete conflicts to avoid
inconsistency.
Attach the V22 version patch set that includes the following changes:
1) merge the V21-0006 into main patches. Instead of reducing the maximum wait
time to 10s, use 30s which is consistent with the wait in slotsync worker.
2) merge the V21-0007 into main patches. To avoid updating the flush too
frequently for each change, it is updated at most per wal_writer_delay which
is consistent with the existing logic in apply worker.
3) Add a new V21-0004 patch to introduce the "max_conflict_retention_duration" option
mentioned above. Thank a lot for Kuroda-San for contributing codes in this
patch.
Best Regards,
Hou zj
Attachments:
v22-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v22-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From 5b962e0cc016006c90d5cb67b05108d943dedd49 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v22 6/6] 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index bde698348f4..7e8c959a2d3 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5888fae2b5..e7f5c2287c9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4308ba38a4e..c019ec2714d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,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 e3e4e41ac38..f08e7276054 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 67d4c29acfa..b552ebc496d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2744,6 +2744,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2760,15 +2763,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2779,7 +2778,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2798,19 +2797,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3131,9 +3139,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3142,20 +3150,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3164,7 +3181,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3175,7 +3192,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 0f5e0a9778d..1b6436324a2 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2131,7 +2131,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2153,17 +2153,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 872cd6e01a3..cc10c3bcb05 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 f8a8d03e533..83ccfd1b504 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"
@@ -670,7 +671,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 37454dc9513..91224b6ea60 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 29580c90710..24a389d7e44 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2146,6 +2146,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,
@@ -2153,7 +2154,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 7a69c13486a..65e4033cb5a 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -113,6 +118,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -120,6 +132,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
v22-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v22-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 9c4006a214621049dea68b3ed50a29c879391b4e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v22 1/6] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 494 ++++++++++++++++++++-
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, 748 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1a..0dcd4afe4cb 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1182,7 +1182,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.
@@ -2356,7 +2360,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2380,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2393,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
@@ -2439,7 +2453,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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7aff..d12320c5ec9 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,
@@ -3573,6 +3648,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
@@ -3651,6 +3727,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);
@@ -3677,6 +3755,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3772,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);
@@ -3706,6 +3810,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)
{
/*
@@ -3740,6 +3849,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,
@@ -3803,6 +3919,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
@@ -3838,7 +3956,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;
@@ -3916,6 +4033,381 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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)
+{
+ XLogRecPtr writepos;
+ XLogRecPtr flushpos;
+ bool have_pending_txes;
+
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * 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))
+ {
+ /* 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 a0782b1bbf6..1299ed1eb5d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 94dc956ae8c..449b12da980 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2477,6 +2477,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v22-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v22-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 4de854db4c1b6eca3b192dabfee8de04b6654d18 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v22 2/6] 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 | 237 +++++++++++++++++-
.../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, 302 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 3f41a17b1fe..4beb276a3f2 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4736,6 +4736,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581ae..feb026d0c5a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea206..d49ba78b13e 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..a842426f5c8 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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();
@@ -1167,14 +1190,49 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
+
+ /*
+ * 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 the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1265,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1319,154 @@ 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);
+ return;
+ }
+
+ /*
+ * Initially create persistent slot as ephemeral - that allows us to
+ * nicely handle errors during initialization because it'll get dropped if
+ * this transaction fails. We'll make it persistent after safely
+ * initializing the xmin of the slot.
+ */
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf0..64b43346a90 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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 d12320c5ec9..dcd9ea83297 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4349,6 +4349,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 b30e0473e1c..3a3884b3820 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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 82b202f3305..7b29f1814db 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 bf62b36ad07..ea0485efa2e 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.
*
@@ -261,7 +268,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);
--
2.30.0.windows.2
v22-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v22-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 134836a62def3353456dfbbc83314596abaea909 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 9 Jan 2025 16:44:53 +0800
Subject: [PATCH v22 3/6] 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 | 46 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 438 insertions(+), 124 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index d3036c5ba9d..14c094e1b5e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8051,6 +8051,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a083..bde698348f4 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2160,7 +2160,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 64a873a16e3..5f3f5d06d38 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1362,7 +1362,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 a842426f5c8..df978f9d4d2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1189,12 +1191,21 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * 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.
+ */
+ if (sub->retainconflictinfo)
{
- can_advance_xmin = false;
- continue;
+ retain_conflict_info = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1214,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;
@@ -1232,7 +1244,8 @@ ApplyLauncherMain(Datum main_arg)
* 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 (retain_conflict_info)
+ create_conflict_slot_if_not_exists();
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1270,7 +1283,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1284,12 +1297,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1320,7 +1335,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
@@ -1366,7 +1381,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1377,7 +1392,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1455,8 +1473,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 dcd9ea83297..f2af1fe861d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4098,6 +4098,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
@@ -4510,6 +4514,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 ||
@@ -4519,7 +4532,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 8f73a5df956..7940dc5803f 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1b..0be12d05c93 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index 2ef99971ac0..54a89d5d62c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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)
{
@@ -6733,6 +6733,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 6194b757d59..1d1e0dffb9e 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v22-0004-add-a-max_conflict_retention_duration-subscripti.patchapplication/octet-stream; name=v22-0004-add-a-max_conflict_retention_duration-subscripti.patchDownload
From 368a68aa6894a2dbcfa5904a83170efb4636db32 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 15 Jan 2025 16:46:15 +0800
Subject: [PATCH v22 4/6] add a max_conflict_retention_duration subscription
option
This commit introduces the max_conflict_retention_duration subscription option,
designed to prevent excessive accumulation of dead tuples when
retain_conflict_info is enabled 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 forcibly advances
the ID. Consequently, this causes the xmin of the replication slot
pg_conflict_detection to advance, making the dead tuples removable.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 25 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/worker.c | 30 +++-
src/bin/pg_dump/pg_dump.c | 19 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 16 +-
src/include/catalog/pg_subscription.h | 10 ++
src/test/regress/expected/subscription.out | 193 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
13 files changed, 255 insertions(+), 105 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e5415c3150d..b772cefe075 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 61374b74d1c..09988a56866 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -466,6 +466,31 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Specifies the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 39cfae43d6f..d3d9cc1779c 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retainconflictinfo = subform->subretainconflictinfo;
+ sub->maxconflictretentionduration = subform->submaxconflictretentionduration;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5f3f5d06d38..4308ba38a4e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1362,8 +1362,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretainconflictinfo, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretainconflictinfo, submaxconflictretentionduration, 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 51bd52e48a3..13199740320 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retainconflictinfo;
+ int32 maxconflictretentionduration;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO))
opts->retainconflictinfo = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflictretentionduration = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO;
opts->retainconflictinfo = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflictretentionduration = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -695,6 +709,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretainconflictinfo - 1] =
BoolGetDatum(opts.retainconflictinfo);
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1193,7 +1209,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1376,6 +1393,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_conflict_info = opts.retainconflictinfo;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
+ replaces[Anum_pg_subscription_submaxconflictretentionduration - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f2af1fe861d..67d4c29acfa 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4336,9 +4336,35 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise,
+ * get the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, forcibly advance the non-removable
+ * transaction ID.
+ */
+ if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ return;
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" is forcibly advancing non-removable transaction ID to %u",
+ MySubscription->name,
+ XidFromFullTransactionId(data->candidate_xid)),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ MySubscription->maxconflictretentionduration));
+ }
/*
* Reaching here means the remote WAL position has been received, and all
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 7940dc5803f..14a8691a4eb 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4868,6 +4868,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretainconflictinfo;
+ int i_submaxconflictretentionduration;
int i,
ntups;
@@ -4947,10 +4948,18 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 180000)
appendPQExpBufferStr(query,
- " s.subretainconflictinfo\n");
+ " s.subretainconflictinfo,\n");
else
appendPQExpBuffer(query,
- " false AS subretainconflictinfo\n");
+ " false AS subretainconflictinfo,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflictretentionduration\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflictretentionduration\n");
+
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4984,6 +4993,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo");
+ i_submaxconflictretentionduration = PQfnumber(res, "submaxconflictretentionduration");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5019,6 +5029,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretainconflictinfo =
(strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0);
+ subinfo[i].submaxconflictretentionduration =
+ atoi(PQgetvalue(res, i, i_submaxconflictretentionduration));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5280,6 +5292,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretainconflictinfo)
appendPQExpBufferStr(query, ", retain_conflict_info = true");
+ if (subinfo->submaxconflictretentionduration)
+ appendPQExpBuffer(query, ", submaxconflictretentionduration = %d", subinfo->submaxconflictretentionduration);
+
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 0be12d05c93..f58716b5a74 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretainconflictinfo;
+ int submaxconflictretentionduration;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 54a89d5d62c..25571f4c68b 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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, false, false};
if (pset.sversion < 100000)
{
@@ -6734,10 +6734,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 180000)
+ {
appendPQExpBuffer(&buf,
", subretainconflictinfo AS \"%s\"\n",
gettext_noop("Retain conflict info"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflictretentionduration AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index e0be3c6fd63..d899428e5d5 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,10 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin", "password_required",
+ "retain_conflict_info", "run_as_owner", "slot_name",
+ "streaming", "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,10 +3689,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin", "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 1d1e0dffb9e..49b6cf2d36e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,12 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretainconflictinfo; /* True if information useful for
* conflict detection is retained */
+ int32 submaxconflictretentionduration; /* The maximum duration
+ * (in milliseconds) for
+ * which conflict
+ * information can be
+ * retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +142,10 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retainconflictinfo; /* True if information useful for conflict
* detection is retained */
+ int32 maxconflictretentionduration; /* The maximum duration (in
+ * milliseconds) for which
+ * conflict information can be
+ * 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 bff4cc051db..577493fe17b 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -434,19 +434,44 @@ WARNING: information for detecting conflicts cannot be fully retained when "tra
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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | 0 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 c65397e5ac6..17dd6aa26a9 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -303,6 +303,22 @@ ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.30.0.windows.2
v22-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v22-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 446b210a333a7e1d3a1a0b22df1d05647d174470 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v22 5/6] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 156 ++++++++++++++++++
2 files changed, 157 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..7a69c13486a
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,156 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
On Wednesday, January 15, 2025 4:51 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
On Wednesday, January 15, 2025 12:08 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:Hi,
On Wed, Jan 15, 2025 at 5:57 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:On Mon, Jan 13, 2025 at 8:39 PM Amit Kapila
<amit.kapila16@gmail.com>wrote:
As of now, I can't think of a way to throttle the publisher when
the apply_worker lags. Basically, we need some way to throttle
(reduce the speed of backends) when the apply worker is lagging
behind a threshold margin. Can you think of some way? I thought if
one notices frequent invalidation of the launcher's slot due to
max_lag, then they can rebalance their workload on the publisher.I don't have any ideas other than invalidating the launcher's slot
when the apply lag is huge. We can think of invalidating the
launcher's slot for some reasons such as the replay lag, the age of
slot's xmin, and the duration.Right, this is exactly where we are heading. I think we can add
reasons step-wise. For example, as a first step, we can invalidate the
slot due to replay LAG. Then, slowly, we can add other reasons as well.One thing that needs more discussion is the exact way to invalidate a
slot. I have mentioned a couple of ideas in my previous email which I
am writing
again: "If we just invalidate the slot, users can check the status of
the slot and need to disable/enable retain_conflict_info again to
start retaining the required information. This would be required
because we can't allow system slots (slots created
internally) to be created by users. The other way could be that
instead of invalidating the slot, we directly drop/re-create the slot
or increase its xmin. If we choose to advance the slot automatically
without user intervention, we need to let users know via LOG and or viainformation in the view."
In the latest version, we implemented a simpler approach that allows the apply
worker to directly advance the oldest_nonremovable_xid if the waiting time
exceeds the newly introduced option's limit. I've named this option
"max_conflict_retention_duration," as it aligns better with the conflict
detection concept and the "retain_conflict_info" option.During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker
evaluates how much time it has spent waiting. If this duration exceeds the
max_conflict_retention_duration, the worker directly advances the
oldest_nonremovable_xid and logs a message indicating the forced
advancement of the non-removable transaction ID.This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler because
we can avoid adding 1) new slot invalidation type due to apply lag, 2) new field
lag_behind in shared memory (MyLogicalRepWorker) to indicate when the lag
exceeds the limit, and 3) additional logic in the launcher to handle each
worker's lag status.In the slot invalidation, user would be able to confirm if the current by checking
if the slot in pg_replication_slot in invalidated or not, while in the simpler
approach mentioned, user could only confirm that by checking the LOGs.What do you think ? If we prefer the slot invalidation approach, I can do that part
in next version.The max_lag idea sounds interesting for the case
where the subscriber is much behind. Probably we can visit this
idea as a new feature after completing this feature.Sure, but what will be our answer to users for cases where the
performance tanks due to bloat accumulation? The tests show that
once the apply_lag becomes large, it becomes almost impossible for
the apply worker to catch up (or take a very long time) and
advance the slot's xmin. The users can disable
retain_conflict_info to bring back the performance and get rid of
bloat but I thought it would be easier for users to do that if we
have some knob where they don't need to wait till actually the problem ofbloat/performance dip happens.
Probably retaining dead tuples based on the time duration or its age
might be other solutions, it would increase a risk of not being able
to detect update_deleted conflict though. I think in any way as long
as we accumulate dead tulpes to detect update_deleted conflicts, it
would be a tradeoff between reliably detecting update_deleted
conflicts and the performance.Right, and users have an option for it. Say, if they set max_lag as -1
(or some special value), we won't invalidate the slot, so the
update_delete conflict can be detected with complete reliability. At
this stage, it is okay if this information is LOGGED and displayed via
a system view. We need more thoughts while working on the CONFLICT
RESOLUTION patch such as we may need to additionally display a WARNING
or ERROR if the remote_tuples commit_time is earlier than the last
time slot is invalidated. I don't want to go in a detailed discussion
at this point but just wanted you to know that we will need additional
work for the resolution of update_delete conflicts to avoid inconsistency.Attach the V22 version patch set that includes the following changes:
1) merge the V21-0006 into main patches. Instead of reducing the maximum
wait
time to 10s, use 30s which is consistent with the wait in slotsync worker.
2) merge the V21-0007 into main patches. To avoid updating the flush too
frequently for each change, it is updated at most per wal_writer_delay
which
is consistent with the existing logic in apply worker.
3) Add a new V21-0004 patch to introduce the
"max_conflict_retention_duration" option
mentioned above. Thank a lot for Kuroda-San for contributing codes in this
patch.
CFbot reported a test error[1]https://cirrus-ci.com/task/5635566210908160 which is because I missed to check if the
new option value is 0 before checking the time. Here is a new version
patch set to fix this issue.
[1]: https://cirrus-ci.com/task/5635566210908160
Best Regards,
Hou zj
Attachments:
v23-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v23-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From 4af794b5dc8232a4d10e64fb7f63712b6ebbd90e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v23 6/6] 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index bde698348f4..7e8c959a2d3 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1602,6 +1602,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5888fae2b5..e7f5c2287c9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4308ba38a4e..c019ec2714d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,6 +1373,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 e3e4e41ac38..f08e7276054 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 c83dd44c94e..8f247075119 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2744,6 +2744,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2760,15 +2763,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2779,7 +2778,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2798,19 +2797,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3131,9 +3139,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3142,20 +3150,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3164,7 +3181,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3175,7 +3192,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 0f5e0a9778d..1b6436324a2 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2131,7 +2131,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2153,17 +2153,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 872cd6e01a3..cc10c3bcb05 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 f8a8d03e533..83ccfd1b504 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"
@@ -670,7 +671,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 37454dc9513..91224b6ea60 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 29580c90710..24a389d7e44 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2146,6 +2146,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,
@@ -2153,7 +2154,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 7a69c13486a..65e4033cb5a 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -113,6 +118,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -120,6 +132,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
v23-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v23-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 740b6351873e5d191ecb568e59fc2d6101efb22b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 14:11:40 +0800
Subject: [PATCH v23 1/4] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 494 ++++++++++++++++++++-
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, 748 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1a..0dcd4afe4cb 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1182,7 +1182,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.
@@ -2356,7 +2360,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2380,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2393,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
@@ -2439,7 +2453,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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7aff..99ec14cbff9 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,
@@ -3573,6 +3648,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
@@ -3651,6 +3727,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);
@@ -3677,6 +3755,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3772,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);
@@ -3706,6 +3810,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)
{
/*
@@ -3740,6 +3849,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,
@@ -3803,6 +3919,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
@@ -3838,7 +3956,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;
@@ -3916,6 +4033,381 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * 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 a0782b1bbf6..1299ed1eb5d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 94dc956ae8c..449b12da980 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2477,6 +2477,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
RevmapContents
--
2.30.0.windows.2
v23-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v23-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 062f09354c56e69d86c730f0a30fb076714ed664 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v23 2/4] 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 | 237 +++++++++++++++++-
.../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, 302 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 3f41a17b1fe..4beb276a3f2 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4736,6 +4736,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581ae..feb026d0c5a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea206..d49ba78b13e 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..a842426f5c8 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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();
@@ -1167,14 +1190,49 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
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 */
+ }
+
+ /*
+ * 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 the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1265,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1319,154 @@ 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);
+ return;
+ }
+
+ /*
+ * Initially create persistent slot as ephemeral - that allows us to
+ * nicely handle errors during initialization because it'll get dropped if
+ * this transaction fails. We'll make it persistent after safely
+ * initializing the xmin of the slot.
+ */
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf0..64b43346a90 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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 99ec14cbff9..02ecc3c5f5a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4349,6 +4349,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 b30e0473e1c..3a3884b3820 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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 82b202f3305..7b29f1814db 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 bf62b36ad07..ea0485efa2e 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.
*
@@ -261,7 +268,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);
--
2.30.0.windows.2
v23-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v23-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 3c2d3bee138271d977a17014c37ae55b2313b663 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 9 Jan 2025 16:44:53 +0800
Subject: [PATCH v23 3/4] 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 | 46 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 438 insertions(+), 124 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index d3036c5ba9d..14c094e1b5e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8051,6 +8051,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 8290cd1a083..bde698348f4 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2160,7 +2160,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 64a873a16e3..5f3f5d06d38 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1362,7 +1362,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 a842426f5c8..df978f9d4d2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1189,12 +1191,21 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- if (!sub->enabled)
+ /*
+ * 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.
+ */
+ if (sub->retainconflictinfo)
{
- can_advance_xmin = false;
- continue;
+ retain_conflict_info = true;
+ can_advance_xmin &= sub->enabled;
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1203,10 +1214,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;
@@ -1232,7 +1244,8 @@ ApplyLauncherMain(Datum main_arg)
* 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 (retain_conflict_info)
+ create_conflict_slot_if_not_exists();
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1270,7 +1283,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1284,12 +1297,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1320,7 +1335,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
@@ -1366,7 +1381,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1377,7 +1392,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1455,8 +1473,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 02ecc3c5f5a..79d3e1683ed 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4098,6 +4098,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
@@ -4510,6 +4514,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 ||
@@ -4519,7 +4532,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 8f73a5df956..7940dc5803f 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1b..0be12d05c93 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index 2ef99971ac0..54a89d5d62c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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)
{
@@ -6733,6 +6733,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 6194b757d59..1d1e0dffb9e 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v23-0004-add-a-max_conflict_retention_duration-subscripti.patchapplication/octet-stream; name=v23-0004-add-a-max_conflict_retention_duration-subscripti.patchDownload
From dbf5456780cbd27a77c1f2e7f04a35e26cf2073c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 15 Jan 2025 16:46:15 +0800
Subject: [PATCH v23 4/4] add a max_conflict_retention_duration subscription
option
This commit introduces the max_conflict_retention_duration subscription option,
designed to prevent excessive accumulation of dead tuples when
retain_conflict_info is enabled 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 forcibly advances
the ID. Consequently, this causes the xmin of the replication slot
pg_conflict_detection to advance, making the dead tuples removable.
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 25 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/worker.c | 31 +++-
src/bin/pg_dump/pg_dump.c | 19 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 16 +-
src/include/catalog/pg_subscription.h | 10 ++
src/test/regress/expected/subscription.out | 193 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
14 files changed, 266 insertions(+), 105 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 14c094e1b5e..3682a6c3137 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8063,6 +8063,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflictretentionduration</structfield> <type>interger</type>
+ </para>
+ <para>
+ If not zero, the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e5415c3150d..b772cefe075 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 61374b74d1c..01abe7bbba8 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -466,6 +466,31 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Specifies the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 39cfae43d6f..d3d9cc1779c 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retainconflictinfo = subform->subretainconflictinfo;
+ sub->maxconflictretentionduration = subform->submaxconflictretentionduration;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5f3f5d06d38..4308ba38a4e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1362,8 +1362,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretainconflictinfo, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretainconflictinfo, submaxconflictretentionduration, 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 51bd52e48a3..13199740320 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retainconflictinfo;
+ int32 maxconflictretentionduration;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO))
opts->retainconflictinfo = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflictretentionduration = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO;
opts->retainconflictinfo = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflictretentionduration = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -695,6 +709,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretainconflictinfo - 1] =
BoolGetDatum(opts.retainconflictinfo);
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1193,7 +1209,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1376,6 +1393,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_conflict_info = opts.retainconflictinfo;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
+ replaces[Anum_pg_subscription_submaxconflictretentionduration - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 79d3e1683ed..01cf9221cc0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4336,9 +4336,36 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise,
+ * get the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, forcibly advance the non-removable
+ * transaction ID.
+ */
+ if (!MySubscription->maxconflictretentionduration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ return;
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" is forcibly advancing non-removable transaction ID to %u",
+ MySubscription->name,
+ XidFromFullTransactionId(data->candidate_xid)),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ MySubscription->maxconflictretentionduration));
+ }
/*
* Reaching here means the remote WAL position has been received, and all
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 7940dc5803f..14a8691a4eb 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4868,6 +4868,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretainconflictinfo;
+ int i_submaxconflictretentionduration;
int i,
ntups;
@@ -4947,10 +4948,18 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 180000)
appendPQExpBufferStr(query,
- " s.subretainconflictinfo\n");
+ " s.subretainconflictinfo,\n");
else
appendPQExpBuffer(query,
- " false AS subretainconflictinfo\n");
+ " false AS subretainconflictinfo,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflictretentionduration\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflictretentionduration\n");
+
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4984,6 +4993,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo");
+ i_submaxconflictretentionduration = PQfnumber(res, "submaxconflictretentionduration");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5019,6 +5029,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretainconflictinfo =
(strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0);
+ subinfo[i].submaxconflictretentionduration =
+ atoi(PQgetvalue(res, i, i_submaxconflictretentionduration));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5280,6 +5292,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretainconflictinfo)
appendPQExpBufferStr(query, ", retain_conflict_info = true");
+ if (subinfo->submaxconflictretentionduration)
+ appendPQExpBuffer(query, ", submaxconflictretentionduration = %d", subinfo->submaxconflictretentionduration);
+
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 0be12d05c93..f58716b5a74 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretainconflictinfo;
+ int submaxconflictretentionduration;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 54a89d5d62c..25571f4c68b 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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, false, false};
if (pset.sversion < 100000)
{
@@ -6734,10 +6734,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 180000)
+ {
appendPQExpBuffer(&buf,
", subretainconflictinfo AS \"%s\"\n",
gettext_noop("Retain conflict info"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflictretentionduration AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index e0be3c6fd63..d899428e5d5 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,10 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin", "password_required",
+ "retain_conflict_info", "run_as_owner", "slot_name",
+ "streaming", "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,10 +3689,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin", "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 1d1e0dffb9e..49b6cf2d36e 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,12 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretainconflictinfo; /* True if information useful for
* conflict detection is retained */
+ int32 submaxconflictretentionduration; /* The maximum duration
+ * (in milliseconds) for
+ * which conflict
+ * information can be
+ * retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +142,10 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retainconflictinfo; /* True if information useful for conflict
* detection is retained */
+ int32 maxconflictretentionduration; /* The maximum duration (in
+ * milliseconds) for which
+ * conflict information can be
+ * 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 bff4cc051db..577493fe17b 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -434,19 +434,44 @@ WARNING: information for detecting conflicts cannot be fully retained when "tra
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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | 0 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 c65397e5ac6..17dd6aa26a9 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -303,6 +303,22 @@ ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.30.0.windows.2
v23-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v23-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From ddcbd09ea7e7d13b9d42b681200478bded7639f5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v23 5/6] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 156 ++++++++++++++++++
2 files changed, 157 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..7a69c13486a
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,156 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
On Wed, Jan 15, 2025 at 2:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
In the latest version, we implemented a simpler approach that allows the apply
worker to directly advance the oldest_nonremovable_xid if the waiting time
exceeds the newly introduced option's limit. I've named this option
"max_conflict_retention_duration," as it aligns better with the conflict
detection concept and the "retain_conflict_info" option.During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker evaluates
how much time it has spent waiting. If this duration exceeds the
max_conflict_retention_duration, the worker directly advances the
oldest_nonremovable_xid and logs a message indicating the forced advancement of
the non-removable transaction ID.This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler because we
can avoid adding 1) new slot invalidation type due to apply lag, 2) new field
lag_behind in shared memory (MyLogicalRepWorker) to indicate when the lag
exceeds the limit, and 3) additional logic in the launcher to handle each
worker's lag status.In the slot invalidation, user would be able to confirm if the current by
checking if the slot in pg_replication_slot in invalidated or not, while in the
simpler approach mentioned, user could only confirm that by checking the LOGs.
The user needs to check the LOGs corresponding to all subscriptions on
the node. I see the simplicity of the approach you used but still the
slot_invalidation idea sounds better to me on the grounds that it will
be convenient for users/DBA to know when to rely on the update_missing
type conflict if there is a valid and active slot with the name
'pg_conflict_detection' (or whatever name we decide to give) then
users can rely on the detected conflict. Sawada-San, and others, do
you have any preference on this matter?
Do we want to prohibit the combination copy_data as true and
retain_conflict_info=true? I understand that with the new parameter
'max_conflict_retention_duration', for large copies slot would anyway
be invalidated but I don't want to give users more ways to see this
slot invalidated in the beginning itself. Similarly during ALTER
SUBSCRIPTION, if the initial synch is in progress, we can disallow
enabling retain_conflict_info. Later, if there is a real demand for
such a combination, we can always enable it.
--
With Regards,
Amit Kapila.
On Fri, Jan 3, 2025 at 4:31 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jan 2, 2025 at 2:57 PM vignesh C <vignesh21@gmail.com> wrote:
Conflict detection of truncated updates is detected as update_missing
and deleted update is detected as update_deleted. I was not sure if
truncated updates should also be detected as update_deleted, as the
document says truncate operation is "It has the same effect as an
unqualified DELETE on each table" at [1].This is expected behavior because TRUNCATE would immediately reclaim
space and remove all the data. So, there is no way to retain the
removed row.
I’m not sure whether to call this expected behavior or simply acknowledge
that we have no way to control it. Logically, it would have been preferable
if it behaved like a DELETE, but we are constrained by the way TRUNCATE
works. At least that's what my opinion about this case.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Thu, Jan 16, 2025 at 3:45 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Jan 3, 2025 at 4:31 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jan 2, 2025 at 2:57 PM vignesh C <vignesh21@gmail.com> wrote:
Conflict detection of truncated updates is detected as update_missing
and deleted update is detected as update_deleted. I was not sure if
truncated updates should also be detected as update_deleted, as the
document says truncate operation is "It has the same effect as an
unqualified DELETE on each table" at [1].This is expected behavior because TRUNCATE would immediately reclaim
space and remove all the data. So, there is no way to retain the
removed row.I’m not sure whether to call this expected behavior or simply acknowledge that we have no way to control it. Logically, it would have been preferable if it behaved like a DELETE, but we are constrained by the way TRUNCATE works.
I see your point. So, it is probably better to add a Note about this.
--
With Regards,
Amit Kapila.
On Thu, Jan 16, 2025 at 4:02 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jan 16, 2025 at 3:45 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Jan 3, 2025 at 4:31 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
On Thu, Jan 2, 2025 at 2:57 PM vignesh C <vignesh21@gmail.com> wrote:
Conflict detection of truncated updates is detected as update_missing
and deleted update is detected as update_deleted. I was not sure if
truncated updates should also be detected as update_deleted, as the
document says truncate operation is "It has the same effect as an
unqualified DELETE on each table" at [1].This is expected behavior because TRUNCATE would immediately reclaim
space and remove all the data. So, there is no way to retain the
removed row.I’m not sure whether to call this expected behavior or simply
acknowledge that we have no way to control it. Logically, it would have
been preferable if it behaved like a DELETE, but we are constrained by the
way TRUNCATE works.I see your point. So, it is probably better to add a Note about this.
+1
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wed, Jan 15, 2025 at 9:38 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 15, 2025 at 5:57 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Probably retaining dead tuples based on the time duration or its age
might be other solutions, it would increase a risk of not being able
to detect update_deleted conflict though. I think in any way as long
as we accumulate dead tulpes to detect update_deleted conflicts, it
would be a tradeoff between reliably detecting update_deleted
conflicts and the performance.Right, and users have an option for it. Say, if they set max_lag as -1
(or some special value), we won't invalidate the slot, so the
update_delete conflict can be detected with complete reliability. At
this stage, it is okay if this information is LOGGED and displayed via
a system view. We need more thoughts while working on the CONFLICT
RESOLUTION patch such as we may need to additionally display a WARNING
or ERROR if the remote_tuples commit_time is earlier than the last
time slot is invalidated.
The more reliable way to do something in this regard would be that if
there is a valid and active pg_conflict_detection (or whatever we name
this slot) then consider the update_missing conflict detected as
reliable. Otherwise, the conflict_type will depend on whether the
vacuum has removed the dead row. so, the conflict management system or
users would easily know when to rely on this update_missing conflict
type.
--
With Regards,
Amit Kapila.
On Thu, Jan 16, 2025 at 2:02 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 15, 2025 at 2:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:In the latest version, we implemented a simpler approach that allows the apply
worker to directly advance the oldest_nonremovable_xid if the waiting time
exceeds the newly introduced option's limit. I've named this option
"max_conflict_retention_duration," as it aligns better with the conflict
detection concept and the "retain_conflict_info" option.During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker evaluates
how much time it has spent waiting. If this duration exceeds the
max_conflict_retention_duration, the worker directly advances the
oldest_nonremovable_xid and logs a message indicating the forced advancement of
the non-removable transaction ID.This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler because we
can avoid adding 1) new slot invalidation type due to apply lag, 2) new field
lag_behind in shared memory (MyLogicalRepWorker) to indicate when the lag
exceeds the limit, and 3) additional logic in the launcher to handle each
worker's lag status.In the slot invalidation, user would be able to confirm if the current by
checking if the slot in pg_replication_slot in invalidated or not, while in the
simpler approach mentioned, user could only confirm that by checking the LOGs.The user needs to check the LOGs corresponding to all subscriptions on
the node. I see the simplicity of the approach you used but still the
slot_invalidation idea sounds better to me on the grounds that it will
be convenient for users/DBA to know when to rely on the update_missing
type conflict if there is a valid and active slot with the name
'pg_conflict_detection' (or whatever name we decide to give) then
users can rely on the detected conflict. Sawada-San, and others, do
you have any preference on this matter?
I also think that it would be convenient for users if they could check
if there was a valid and active pg_conflict_detection slot to know
when to rely on detected conflicts. On the other hand, I think it
would not be convenient for users if we always required user
intervention to re-create the slot. Once the slot is invalidated or
dropped, we can no longer guarantee that update_deleted conflicts are
detected reliably, but the logical replication would still be running.
That means we might have already been missing update_deleted
conflicts. From the user perspective, it would be cumbersome to
disable/enable retain_conflict_info (and check if the slot was
re-created) just to make retain_conflict_info work again.
Do we want to prohibit the combination copy_data as true and
retain_conflict_info=true? I understand that with the new parameter
'max_conflict_retention_duration', for large copies slot would anyway
be invalidated but I don't want to give users more ways to see this
slot invalidated in the beginning itself. Similarly during ALTER
SUBSCRIPTION, if the initial synch is in progress, we can disallow
enabling retain_conflict_info. Later, if there is a real demand for
such a combination, we can always enable it.
Does it mean that whenever users want to start the initial sync they
need to disable reatin_conflict_info on all subscriptions? Which
doesn't seem very convenient.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Fri, Jan 17, 2025 at 1:37 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Jan 16, 2025 at 2:02 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 15, 2025 at 2:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:In the latest version, we implemented a simpler approach that allows the apply
worker to directly advance the oldest_nonremovable_xid if the waiting time
exceeds the newly introduced option's limit. I've named this option
"max_conflict_retention_duration," as it aligns better with the conflict
detection concept and the "retain_conflict_info" option.During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker evaluates
how much time it has spent waiting. If this duration exceeds the
max_conflict_retention_duration, the worker directly advances the
oldest_nonremovable_xid and logs a message indicating the forced advancement of
the non-removable transaction ID.This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler because we
can avoid adding 1) new slot invalidation type due to apply lag, 2) new field
lag_behind in shared memory (MyLogicalRepWorker) to indicate when the lag
exceeds the limit, and 3) additional logic in the launcher to handle each
worker's lag status.In the slot invalidation, user would be able to confirm if the current by
checking if the slot in pg_replication_slot in invalidated or not, while in the
simpler approach mentioned, user could only confirm that by checking the LOGs.The user needs to check the LOGs corresponding to all subscriptions on
the node. I see the simplicity of the approach you used but still the
slot_invalidation idea sounds better to me on the grounds that it will
be convenient for users/DBA to know when to rely on the update_missing
type conflict if there is a valid and active slot with the name
'pg_conflict_detection' (or whatever name we decide to give) then
users can rely on the detected conflict. Sawada-San, and others, do
you have any preference on this matter?I also think that it would be convenient for users if they could check
if there was a valid and active pg_conflict_detection slot to know
when to rely on detected conflicts. On the other hand, I think it
would not be convenient for users if we always required user
intervention to re-create the slot. Once the slot is invalidated or
dropped, we can no longer guarantee that update_deleted conflicts are
detected reliably, but the logical replication would still be running.
That means we might have already been missing update_deleted
conflicts. From the user perspective, it would be cumbersome to
disable/enable retain_conflict_info (and check if the slot was
re-created) just to make retain_conflict_info work again.
True, ideally, we can recreate the slot automatically or use the idea
of directly advancing oldest_nonremovable_xid as Hou-San proposed or
directly advance slot's xmin. However, we won't be able to detect
update_delete conflict until the publisher's load is adjusted (or
reduced) because apply worker will keep lagging till that point, even
if we advance slot's xmin automatically. So, we will keep re-creating
the slot or advancing it at regular intervals
(max_conflict_retention_duration) without any additional reliability.
This will lead to bloat retention and or performance dip on subscriber
workload without even one could detect the update_missing type of
conflict reliably.
The other possibilities to avoid/reduce user intervention could be
that once the subscriber catches up with the publisher in terms of
applying WAL, we could re-create/advance the slot. We could do this in
multiple ways, (a) say when last_received_pos from publishers equals
or last_flush_pos on the subscriber, or (b) the apply worker keeps
doing the xid advancement phases but do not actually advance xid, it's
only intended to check the latest lag. If the lag becomes less than
the max_conflict_retention_duration, then notify the launcher to
re-create the slot.
I feel these are some optimizations that could reduce the need to
re-enable retain_conflict_info but users can still do it manually if
they wish to.
Do we want to prohibit the combination copy_data as true and
retain_conflict_info=true? I understand that with the new parameter
'max_conflict_retention_duration', for large copies slot would anyway
be invalidated but I don't want to give users more ways to see this
slot invalidated in the beginning itself. Similarly during ALTER
SUBSCRIPTION, if the initial synch is in progress, we can disallow
enabling retain_conflict_info. Later, if there is a real demand for
such a combination, we can always enable it.Does it mean that whenever users want to start the initial sync they
need to disable reatin_conflict_info on all subscriptions? Which
doesn't seem very convenient.
I agree it is inconvenient but OTOH, if it leads to a large copy then
anyway, the slot may not be able to progress leading to invalidation.
As it is difficult to predict, we may allow it but document that large
copies could lead to slot_invalidation as during that time there is a
possibility that we may not be able to apply any WAL.
--
With Regards,
Amit Kapila.
On Thursday, January 16, 2025 6:02 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 15, 2025 at 2:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:In the latest version, we implemented a simpler approach that allows
the apply worker to directly advance the oldest_nonremovable_xid if
the waiting time exceeds the newly introduced option's limit. I've
named this option "max_conflict_retention_duration," as it aligns
better with the conflict detection concept and the "retain_conflict_info"option.
During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker
evaluates how much time it has spent waiting. If this duration exceeds
the max_conflict_retention_duration, the worker directly advances the
oldest_nonremovable_xid and logs a message indicating the forced
advancement of the non-removable transaction ID.This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler
because we can avoid adding 1) new slot invalidation type due to apply
lag, 2) new field lag_behind in shared memory (MyLogicalRepWorker) to
indicate when the lag exceeds the limit, and 3) additional logic in
the launcher to handle each worker's lag status.In the slot invalidation, user would be able to confirm if the current
by checking if the slot in pg_replication_slot in invalidated or not,
while in the simpler approach mentioned, user could only confirm that bychecking the LOGs.
The user needs to check the LOGs corresponding to all subscriptions on the
node. I see the simplicity of the approach you used but still the
slot_invalidation idea sounds better to me on the grounds that it will be
convenient for users/DBA to know when to rely on the update_missing type
conflict if there is a valid and active slot with the name 'pg_conflict_detection'
(or whatever name we decide to give) then users can rely on the detected
conflict. Sawada-San, and others, do you have any preference on this matter?
I think invalidating the slot is OK and we could also let the apply worker to
automatic recovery as suggested in [1]/messages/by-id/CAA4eK1J0W_ebXSSJXb7uM67=KyzrUG1sVz-kCd-zT16A7Obq=g@mail.gmail.com.
Here is the V24 patch set. I modified 0004 patch to implement the slot
Invalidation part. Since the automatic recovery could be an optimization and
the discussion is in progress, I didn't implement that part.
Do we want to prohibit the combination copy_data as true and
retain_conflict_info=true? I understand that with the new parameter
'max_conflict_retention_duration', for large copies slot would anyway be
invalidated but I don't want to give users more ways to see this slot invalidated
in the beginning itself. Similarly during ALTER SUBSCRIPTION, if the initial
synch is in progress, we can disallow enabling retain_conflict_info. Later, if
there is a real demand for such a combination, we can always enable it.
I didn't restrict this case, but I will analyze more to see if it can be improved.
[1]: /messages/by-id/CAA4eK1J0W_ebXSSJXb7uM67=KyzrUG1sVz-kCd-zT16A7Obq=g@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v24-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v24-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From e70141f1661e0758bb65285eb603ae245f032ee6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v24 6/6] 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 05790396755..f1891e9f21f 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1626,6 +1626,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5888fae2b5..e7f5c2287c9 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2186,6 +2186,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4a910d0e0e6..e5c29d155be 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,6 +1378,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 3985e84d3a6..ecb81c202c2 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 4904a26c5e6..1c7e8c72159 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2746,6 +2746,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2762,15 +2765,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2781,7 +2780,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2800,19 +2799,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3133,9 +3141,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3144,20 +3152,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3166,7 +3183,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3177,7 +3194,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 0f5e0a9778d..1b6436324a2 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2131,7 +2131,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2153,17 +2153,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 18560755d26..20ca775f72c 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 c7db6defd3e..8af7ce4ce38 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"
@@ -671,7 +672,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 37454dc9513..91224b6ea60 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 856a8349c50..ee63784f91b 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2146,6 +2146,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,
@@ -2153,7 +2154,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 7a69c13486a..65e4033cb5a 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -113,6 +118,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -120,6 +132,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.31.1
v24-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v24-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From a151499adb806bca2933d44924490e79e2b57142 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v24 1/6] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 494 ++++++++++++++++++++-
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, 748 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1a..0dcd4afe4cb 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1182,7 +1182,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.
@@ -2356,7 +2360,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2380,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2393,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
@@ -2439,7 +2453,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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7aff..99ec14cbff9 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,
@@ -3573,6 +3648,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
@@ -3651,6 +3727,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);
@@ -3677,6 +3755,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3772,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);
@@ -3706,6 +3810,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)
{
/*
@@ -3740,6 +3849,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,
@@ -3803,6 +3919,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
@@ -3838,7 +3956,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;
@@ -3916,6 +4033,381 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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. In this case, confirming the apply and flush
+ * progress across all table sync workers is complex and not worth the
+ * effort.
+ *
+ * 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 a0782b1bbf6..1299ed1eb5d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 d03921a4822..ff1c5aeb3d2 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2478,6 +2478,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
v24-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v24-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From f06c4b5d181757ee233ec28036c352c04a277708 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v24 2/6] 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 | 237 +++++++++++++++++-
.../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, 302 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a8866292d46..54bbc1a901d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4736,6 +4736,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581ae..feb026d0c5a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea206..d49ba78b13e 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..68614dbf4cb 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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 +1189,50 @@ 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 */
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1207,6 +1265,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1319,154 @@ 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);
+ return;
+ }
+
+ /*
+ * Initially create persistent slot as ephemeral - that allows us to
+ * nicely handle errors during initialization because it'll get dropped if
+ * this transaction fails. We'll make it persistent after safely
+ * initializing the xmin of the slot.
+ */
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
+ RS_EPHEMERAL, 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);
+
+ ReplicationSlotPersist();
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf0..64b43346a90 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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 99ec14cbff9..02ecc3c5f5a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4349,6 +4349,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 b30e0473e1c..3a3884b3820 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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 82b202f3305..7b29f1814db 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 bf62b36ad07..ea0485efa2e 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.
*
@@ -261,7 +268,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);
--
2.31.1
v24-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v24-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 258457d14a2d65c2b08a40e0e7007ff1e7743122 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:15:07 +0800
Subject: [PATCH v24 3/6] 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 | 50 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 440 insertions(+), 126 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index d3036c5ba9d..14c094e1b5e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8051,6 +8051,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 7cc5f4b18d6..05790396755 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2184,7 +2184,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 46868bf7e89..c87d8a84d7c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1367,7 +1367,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 68614dbf4cb..6104d20b0d7 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1190,16 +1192,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);
@@ -1209,10 +1221,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;
@@ -1270,7 +1283,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1284,12 +1297,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1320,7 +1335,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
@@ -1366,7 +1381,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1377,7 +1392,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1455,8 +1473,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 02ecc3c5f5a..79d3e1683ed 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4098,6 +4098,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
@@ -4510,6 +4514,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 ||
@@ -4519,7 +4532,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 8f73a5df956..7940dc5803f 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1b..0be12d05c93 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index 2ef99971ac0..54a89d5d62c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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)
{
@@ -6733,6 +6733,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.31.1
v24-0004-add-a-max_conflict_retention_duration-subscripti.patchapplication/octet-stream; name=v24-0004-add-a-max_conflict_retention_duration-subscripti.patchDownload
From 4bdae9d66fc36a42ed2728991af6c1719034d4ed Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 15 Jan 2025 16:46:15 +0800
Subject: [PATCH v24 4/6] add a max_conflict_retention_duration subscription
option
This commit introduces the max_conflict_retention_duration subscription option,
designed to prevent excessive accumulation of dead tuples when
retain_conflict_info is enabled 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.
---
doc/src/sgml/catalogs.sgml | 10 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 38 ++++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/launcher.c | 39 ++++-
src/backend/replication/logical/worker.c | 130 ++++++++++++--
src/backend/replication/slot.c | 12 +-
src/bin/pg_dump/pg_dump.c | 19 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 16 +-
src/include/catalog/pg_subscription.h | 10 ++
src/include/replication/slot.h | 2 +
src/include/replication/worker_internal.h | 6 +
src/test/regress/expected/subscription.out | 193 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 419 insertions(+), 123 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 14c094e1b5e..3682a6c3137 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8063,6 +8063,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflictretentionduration</structfield> <type>interger</type>
+ </para>
+ <para>
+ If not zero, the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e5415c3150d..b772cefe075 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 61374b74d1c..c6d87255a39 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -466,6 +466,44 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Specifies the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 39cfae43d6f..d3d9cc1779c 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retainconflictinfo = subform->subretainconflictinfo;
+ sub->maxconflictretentionduration = subform->submaxconflictretentionduration;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c87d8a84d7c..4a910d0e0e6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1367,8 +1367,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretainconflictinfo, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretainconflictinfo, submaxconflictretentionduration, 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 51bd52e48a3..13199740320 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retainconflictinfo;
+ int32 maxconflictretentionduration;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO))
opts->retainconflictinfo = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflictretentionduration = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO;
opts->retainconflictinfo = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflictretentionduration = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -695,6 +709,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretainconflictinfo - 1] =
BoolGetDatum(opts.retainconflictinfo);
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1193,7 +1209,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1376,6 +1393,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_conflict_info = opts.retainconflictinfo;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
+ replaces[Anum_pg_subscription_submaxconflictretentionduration - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 6104d20b0d7..87e49c077ed 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -458,6 +458,7 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_retention = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1170,7 +1171,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();
@@ -1199,7 +1201,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1225,12 +1227,13 @@ 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;
SpinLockAcquire(&w->relmutex);
nonremovable_xid = w->oldest_nonremovable_xid;
+ nstop_retention += w->stop_conflict_retention ? 1 : 0;
SpinLockRelease(&w->relmutex);
/*
@@ -1239,8 +1242,9 @@ ApplyLauncherMain(Datum main_arg)
*/
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;
}
@@ -1278,12 +1282,35 @@ 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, and update the sleep time before the next
* attempt.
*/
- if (retain_conflict_info)
+ else if (nretain_conflict_info)
{
bool updated = false;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 79d3e1683ed..4904a26c5e6 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"
@@ -453,6 +454,7 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -4110,6 +4112,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_retention)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4279,6 +4285,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ bool stop_conflict_retention = false;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4336,22 +4344,61 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
- /*
- * 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);
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise,
+ * get the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
- elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
- LSN_FORMAT_ARGS(data->remote_lsn),
- XidFromFullTransactionId(data->candidate_xid));
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (!MySubscription->maxconflictretentionduration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ return;
+
+ stop_conflict_retention = true;
+ }
+
+ if (stop_conflict_retention)
+ {
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ MyLogicalRepWorker->stop_conflict_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ MySubscription->maxconflictretentionduration));
+ }
+ else
+ {
+ /*
+ * 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));
+ }
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4415,6 +4462,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_retention = cause != RS_INVAL_NONE;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4586,6 +4678,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();
}
/*
@@ -5224,6 +5326,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 3a3884b3820..b74049b61b4 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -108,10 +108,11 @@ const char *const SlotInvalidationCauses[] = {
[RS_INVAL_WAL_REMOVED] = "wal_removed",
[RS_INVAL_HORIZON] = "rows_removed",
[RS_INVAL_WAL_LEVEL] = "wal_level_insufficient",
+ [RS_INVAL_CONFLICT_RETENTION_DURATION] = "conflict_retention_exceeds_max_duration",
};
/* Maximum number of invalidation causes */
-#define RS_INVAL_MAX_CAUSES RS_INVAL_WAL_LEVEL
+#define RS_INVAL_MAX_CAUSES RS_INVAL_CONFLICT_RETENTION_DURATION
StaticAssertDecl(lengthof(SlotInvalidationCauses) == (RS_INVAL_MAX_CAUSES + 1),
"array length mismatch");
@@ -1568,6 +1569,11 @@ ReportSlotInvalidation(ReplicationSlotInvalidationCause cause,
case RS_INVAL_WAL_LEVEL:
appendStringInfoString(&err_detail, _("Logical decoding on standby requires \"wal_level\" >= \"logical\" on the primary server."));
break;
+
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ appendStringInfo(&err_detail, _("The duration for retaining conflict information exceeds the maximum limit."));
+ break;
+
case RS_INVAL_NONE:
pg_unreachable();
}
@@ -1681,6 +1687,10 @@ InvalidatePossiblyObsoleteSlot(ReplicationSlotInvalidationCause cause,
if (SlotIsLogical(s))
invalidation_cause = cause;
break;
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ if (IsReservedSlotName(NameStr(s->data.name)))
+ invalidation_cause = cause;
+ break;
case RS_INVAL_NONE:
pg_unreachable();
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 7940dc5803f..14a8691a4eb 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4868,6 +4868,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretainconflictinfo;
+ int i_submaxconflictretentionduration;
int i,
ntups;
@@ -4947,10 +4948,18 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 180000)
appendPQExpBufferStr(query,
- " s.subretainconflictinfo\n");
+ " s.subretainconflictinfo,\n");
else
appendPQExpBuffer(query,
- " false AS subretainconflictinfo\n");
+ " false AS subretainconflictinfo,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflictretentionduration\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflictretentionduration\n");
+
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4984,6 +4993,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo");
+ i_submaxconflictretentionduration = PQfnumber(res, "submaxconflictretentionduration");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5019,6 +5029,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretainconflictinfo =
(strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0);
+ subinfo[i].submaxconflictretentionduration =
+ atoi(PQgetvalue(res, i, i_submaxconflictretentionduration));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5280,6 +5292,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretainconflictinfo)
appendPQExpBufferStr(query, ", retain_conflict_info = true");
+ if (subinfo->submaxconflictretentionduration)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflictretentionduration);
+
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 0be12d05c93..f58716b5a74 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretainconflictinfo;
+ int submaxconflictretentionduration;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 54a89d5d62c..25571f4c68b 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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, false, false};
if (pset.sversion < 100000)
{
@@ -6734,10 +6734,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 180000)
+ {
appendPQExpBuffer(&buf,
", subretainconflictinfo AS \"%s\"\n",
gettext_noop("Retain conflict info"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflictretentionduration AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index e0be3c6fd63..d899428e5d5 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,10 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin", "password_required",
+ "retain_conflict_info", "run_as_owner", "slot_name",
+ "streaming", "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,10 +3689,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin", "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 0ac7c0b120c..f5677b53fad 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,12 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretainconflictinfo; /* True if information useful for
* conflict detection is retained */
+ int32 submaxconflictretentionduration; /* The maximum duration
+ * (in milliseconds) for
+ * which conflict
+ * information can be
+ * retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +142,10 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retainconflictinfo; /* True if information useful for conflict
* detection is retained */
+ int32 maxconflictretentionduration; /* The maximum duration (in
+ * milliseconds) for which
+ * conflict information can be
+ * 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/include/replication/slot.h b/src/include/replication/slot.h
index ea0485efa2e..70be49d97f6 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -63,6 +63,8 @@ typedef enum ReplicationSlotInvalidationCause
RS_INVAL_HORIZON,
/* wal_level insufficient for slot */
RS_INVAL_WAL_LEVEL,
+ /* duration of conflict info retention exceeds the maximum limit */
+ RS_INVAL_CONFLICT_RETENTION_DURATION,
} ReplicationSlotInvalidationCause;
extern PGDLLIMPORT const char *const SlotInvalidationCauses[];
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b09486017f4..40469f2df28 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_retention;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index bff4cc051db..577493fe17b 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -434,19 +434,44 @@ WARNING: information for detecting conflicts cannot be fully retained when "tra
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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | 0 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 c65397e5ac6..17dd6aa26a9 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -303,6 +303,22 @@ ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.31.1
v24-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v24-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From c9d1b99d7a3847248d22a207ab82f24796a239e6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v24 5/6] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 156 ++++++++++++++++++
2 files changed, 157 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..7a69c13486a
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,156 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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");
+
+###############################################################################
+# 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);
+
+my $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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.31.1
Dear hackers,
I've created a new script which simulates that user reduce the workload on the
publisher side. Attached zip file contains a script, execution log and pgbench
outputs. Experiments were done with v24 patch set.
Abstract
======
In this test the conflict slot could be invalidated as expected when the workload
on the publisher was high, and it would not get invalidated anymore after reducing
the workload. This shows even if the slot has been invalidated once, users can
continue to detect the update_deleted conflict by reduce the workload on the publisher.
Also, the transaction per second on the subscriber side can be mostly same as
retain_conflict_info = off case after reducing workload on the pub.
Workload
========
v23_measure.sh is a script I used. It is bit complex but mostly done below things:
1. Construct a pub-sub replication system.
2. Run a pgbench (tcp-b like workload) on both nodes. Initially the parallelism
of pgbench is 30 on both nodes. While running the benchmark TPS has been replicated
once per 1 second.
3. Check the status of the conflict slot periodically.
4. If the conflict slot is invalidated, stop the pgbench for both nodes.
5. Disable the retain_conflict_info option and wait until the conflict slot is dropped.
6. Wait until all the changes on the publisher is replicated to the subscriber.
7. Enable the retain_conflict_info and wait until the conflict slot is created.
8. Re-run the pgbench on both nodes. At that time, the parallelism for the publisher
side is cut in half.
9. loop step 3-8 until the total benchmark time becomes 900s.
Parameters
==========
Publisher GUCs:
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
wal_level = logical
Subscriber GUCs:
autovacuum_naptime = '30s'
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
track_commit_timestamp = on
max_conflict_retention_duration is varied twice, 60s and 120s.
Results for max_conflict_retention_duration = 60s
================================
Parallelism of the publisher side is reduced till 30->15->7->3 and finally the
conflict slot is not invalidated. Below tables show 1) parallelism of the bgbench run,
2) time period for the parallelism, and 3) observed TPS of each iterations.
Publisher side
nclients Ran duration (s) TPS
30 80 34587.9
15 83 19148.2
7 87 9609.1
3 647 4120.7
subscriber side
nclients Ran duration (s) TPS
30 80 10688
30 83 10834
30 87 12327.5
30 647 33300.1
For 30/15/7 cases, the conflict slot has been invalidated around 80s later, but
it can survive for parallelism = 3. At that time the TPS on the subscriber side
becomes mostly same as the publisher (nclients=30).
Results for max_conflict_retention_duration = 120s
=================================
The trend was mostly same as 60s case.
Publisher side
nclients Ran duration TPS
30 155 28979.3
15 157 19333.9
7 196 9875.2
3 389 4539
subscriber side
nclients Ran duration TPS
30 155 5925
30 157 6912
30 196 9157.1
30 389 35736.6
Noticed
=====
While creating the script, I found that step 6 (Wait until all the changes on the
publisher is replicated to the subscriber) was necessary. If it was skipped,
the slot would be invalidated soon. This is because the remained changes are not
replicated to the subscriber side yet and the catchup is delayed due to them.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Attachments:
On Wed, Jan 8, 2025 at 4:03 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Wed, Jan 8, 2025 at 1:53 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 8, 2025 at 3:02 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Dec 19, 2024 at 11:11 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
Here is further performance test analysis with v16 patch-set.
- The performance is reduced on the subscriber side (TPS reduction (~50%) [3] ) due
to dead tuple retention for the conflict detection when detect_update_deleted=on.
- Performance reduction happens only on the subscriber side, as workload on the publisher is
pretty high and the apply workers must wait for the amount of transactions with earlier timestamps to
be applied and flushed before advancing the non-removable XID to remove dead tuples.Assuming that the performance dip happened due to dead tuple retention
for the conflict detection, would TPS on other databases also be
affected?As we use slot->xmin to retain dead tuples, shouldn't the impact be
global (means on all databases)?I think so too.
I can also confirm this via performance tests showing a similar TPS
impact on a database when the update_deleted conflict detection is
enabled for a different database on the subscriber node.
~~~~
Setup:
Code : pgHead+v24
Pub-Sub with logical replication and below configurations-
shared_buffers = '30GB'
max_wal_size = 20GB
min_wal_size = 10GB
## configurations only on Sub:
track_commit_timestamp = on
autovacuum_naptime = '30s'
Data on both nodes:
- Pub has pgbench tables created in 'postgres' DB with scale=100. The
tables are with a different name "pgbench_pub_xxx".
- Sub's 'postgres' DB has Pub's tables and is subscribed for all
table changes from the Pub.
- Sub has another DB - 'db1' which has pgbench tables with initial
data of scale=100 and has no subscriptions from Pub.
Test Runs:
- Ran pgbench with 30 clients and 15 minutes each on Pub and Sub's 'db1' DB.
- Collected data for three runs.
(Attached the scripts used for the test.)
Observation:
- At Sub, the TPS on 'db1' impacted despite no subscription created on
this database.
- The TPS is reduced by -84% during 15-minute pgbench runs.
Results:
Run# pubTPS subTPS_db1
1 31760.23258 4865.019909
2 32147.53313 4847.092329
3 31441.95633 4998.91639
Median 31760.23258 4865.019909
pgHead median 30252.22365 30332.64547
regression 5% -84%
~~~~
--
Thanks,
Nisha
On Sat, Jan 18, 2025 at 9:15 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V24 patch set. I modified 0004 patch to implement the slot
Invalidation part. Since the automatic recovery could be an optimization and
the discussion is in progress, I didn't implement that part.
Few comments for patch-0004
====
src/backend/replication/slot.c
1) Need to describe the new macro RS_INVAL_CONFLICT_RETENTION_DURATION
in the comments above InvalidateObsoleteReplicationSlots(), where all
other invalidation causes are explained.
...
* Whether a slot needs to be invalidated depends on the cause. A slot is
* removed if it:
* - RS_INVAL_WAL_REMOVED: requires a LSN older than the given segment
* - RS_INVAL_HORIZON: requires a snapshot <= the given horizon in the given
* db; dboid may be InvalidOid for shared relations
* - RS_INVAL_WAL_LEVEL: is logical
...
2) Can we mention the GUC parameter that defines this "maximum limit"
while reporting?
+
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ appendStringInfo(&err_detail, _("The duration for retaining conflict
information exceeds the maximum limit."));
+ break;
+
Something like -
"The duration for retaining conflict information exceeds the maximum
limit configured in \"%s\".","max_conflict_retention_duration"
=====
doc/src/sgml/ref/create_subscription.sgml
3)
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
Should we add the above info as a “Warning” in the docs?
On Saturday, January 18, 2025 11:45 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Thursday, January 16, 2025 6:02 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:On Wed, Jan 15, 2025 at 2:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:In the latest version, we implemented a simpler approach that allows
the apply worker to directly advance the oldest_nonremovable_xid if
the waiting time exceeds the newly introduced option's limit. I've
named this option "max_conflict_retention_duration," as it aligns
better with the conflict detection concept and the "retain_conflict_info"option.
During the last phase (RCI_WAIT_FOR_LOCAL_FLUSH), the apply worker
evaluates how much time it has spent waiting. If this duration
exceeds the max_conflict_retention_duration, the worker directly
advances the oldest_nonremovable_xid and logs a message indicating
the forced advancement of the non-removable transaction ID.This approach is a bit like a time-based option that discussed before.
Compared to the slot invalidation approach, this approach is simpler
because we can avoid adding 1) new slot invalidation type due to
apply lag, 2) new field lag_behind in shared memory
(MyLogicalRepWorker) to indicate when the lag exceeds the limit, and
3) additional logic in the launcher to handle each worker's lag status.In the slot invalidation, user would be able to confirm if the
current by checking if the slot in pg_replication_slot in
invalidated or not, while in the simpler approach mentioned, user
could only confirm that bychecking the LOGs.
The user needs to check the LOGs corresponding to all subscriptions on
the node. I see the simplicity of the approach you used but still the
slot_invalidation idea sounds better to me on the grounds that it will
be convenient for users/DBA to know when to rely on the update_missing
type conflict if there is a valid and active slot with the name'pg_conflict_detection'
(or whatever name we decide to give) then users can rely on the
detected conflict. Sawada-San, and others, do you have any preference onthis matter?
I think invalidating the slot is OK and we could also let the apply worker to
automatic recovery as suggested in [1].Here is the V24 patch set. I modified 0004 patch to implement the slot
Invalidation part. Since the automatic recovery could be an optimization and the
discussion is in progress, I didn't implement that part.
The implementation is in progress and I will include it in next version.
Here is the V25 patch set that includes the following change:
0001
* Per off-list discussion with Amit, I added few comments to mention the
reason of skipping advancing xid when table sync is in progress and to mention
that the advancement will not be delayed if changes are filtered out on publisher
via row/table filter.
0004
* Fixed a bug that the launcher would advance the slot.xmin when some apply
workers have not yet started.
* Fixed a bug that the launcher did not advance the slot.xmin even if one of the
apply worker has stopped conflict retention due to the lag.
* Add a retain_conflict_info column in the pg_stat_subscription view to
indicate 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. Thanks Kuroda-san for contributing codes
off-list.
Best Regards,
Hou zj
Attachments:
v25-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v25-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 9db94ac33560f98d350fd7a2d3dc589c5ea43241 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v25 1/6] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 498 ++++++++++++++++++++-
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, 752 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index a3190dc4f1a..0dcd4afe4cb 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1182,7 +1182,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.
@@ -2356,7 +2360,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2376,7 +2380,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2389,8 +2393,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
@@ -2439,7 +2453,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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7aff..7ac238cc0a2 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,
@@ -3573,6 +3648,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
@@ -3651,6 +3727,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);
@@ -3677,6 +3755,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3772,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);
@@ -3706,6 +3810,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)
{
/*
@@ -3740,6 +3849,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,
@@ -3803,6 +3919,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
@@ -3838,7 +3956,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;
@@ -3916,6 +4033,385 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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 a0782b1bbf6..1299ed1eb5d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 d03921a4822..ff1c5aeb3d2 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2478,6 +2478,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
v25-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v25-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 70ebe87f92af8ad7fe2c253fa7f520c42da0c2bc Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v25 2/6] 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 | 239 +++++++++++++++++-
.../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, 304 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a8866292d46..54bbc1a901d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4736,6 +4736,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581ae..feb026d0c5a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 0bbe2eea206..d49ba78b13e 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4739,7 +4739,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..e9869664d32 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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 +1189,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 +1271,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1325,150 @@ 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);
+ 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 bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf0..64b43346a90 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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 7ac238cc0a2..0b949d12184 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4353,6 +4353,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 b30e0473e1c..3a3884b3820 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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 82b202f3305..7b29f1814db 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 bf62b36ad07..ea0485efa2e 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.
*
@@ -261,7 +268,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);
--
2.31.1
v25-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v25-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 857134e84f71db21bedc3b67be57855df6f2eab1 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:15:07 +0800
Subject: [PATCH v25 3/6] 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 | 53 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 442 insertions(+), 127 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index d3036c5ba9d..14c094e1b5e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8051,6 +8051,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 7cc5f4b18d6..05790396755 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2184,7 +2184,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 46868bf7e89..c87d8a84d7c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1367,7 +1367,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 e9869664d32..850dd0aaebe 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1190,16 +1192,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);
@@ -1209,10 +1221,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;
@@ -1238,7 +1251,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,
@@ -1276,7 +1290,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1290,12 +1304,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1326,7 +1342,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
@@ -1368,7 +1384,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1379,7 +1395,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1457,8 +1476,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 0b949d12184..1a4c1ae624f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4098,6 +4098,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
@@ -4514,6 +4518,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 ||
@@ -4523,7 +4536,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 8f73a5df956..7940dc5803f 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1b..0be12d05c93 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index 2ef99971ac0..54a89d5d62c 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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)
{
@@ -6733,6 +6733,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.31.1
v25-0004-add-a-max_conflict_retention_duration-subscripti.patchapplication/octet-stream; name=v25-0004-add-a-max_conflict_retention_duration-subscripti.patchDownload
From 267ced8b9e8f138f5fb476c4c8f2f3367043f4ff Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 15 Jan 2025 16:46:15 +0800
Subject: [PATCH v25 4/6] add a max_conflict_retention_duration subscription
option
This commit introduces the max_conflict_retention_duration subscription option,
designed to prevent excessive accumulation of dead tuples when
retain_conflict_info is enabled 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.
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/catalogs.sgml | 10 ++
doc/src/sgml/monitoring.sgml | 13 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 38 ++++
doc/src/sgml/system-views.sgml | 11 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/launcher.c | 79 ++++++++-
src/backend/replication/logical/worker.c | 130 ++++++++++++--
src/backend/replication/slot.c | 12 +-
src/bin/pg_dump/pg_dump.c | 19 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 16 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 10 ++
src/include/replication/slot.h | 2 +
src/include/replication/worker_internal.h | 6 +
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 193 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
22 files changed, 489 insertions(+), 131 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 14c094e1b5e..3682a6c3137 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8063,6 +8063,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflictretentionduration</structfield> <type>interger</type>
+ </para>
+ <para>
+ If not zero, the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5888fae2b5..c1c61e2ed6c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2101,6 +2101,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e5415c3150d..b772cefe075 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 61374b74d1c..c6d87255a39 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -466,6 +466,44 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Specifies the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 8e2b0a7927b..cce6214df50 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -2620,6 +2620,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
perform logical decoding. It is set only for logical slots.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 39cfae43d6f..d3d9cc1779c 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retainconflictinfo = subform->subretainconflictinfo;
+ sub->maxconflictretentionduration = subform->submaxconflictretentionduration;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c87d8a84d7c..523bca04957 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -976,7 +976,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);
@@ -1367,8 +1368,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretainconflictinfo, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretainconflictinfo, submaxconflictretentionduration, 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 51bd52e48a3..13199740320 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retainconflictinfo;
+ int32 maxconflictretentionduration;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO))
opts->retainconflictinfo = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflictretentionduration = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO;
opts->retainconflictinfo = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflictretentionduration = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -695,6 +709,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretainconflictinfo - 1] =
BoolGetDatum(opts.retainconflictinfo);
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1193,7 +1209,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1376,6 +1393,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_conflict_info = opts.retainconflictinfo;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
+ replaces[Anum_pg_subscription_submaxconflictretentionduration - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 850dd0aaebe..7a3f1c48455 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"
@@ -42,6 +43,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
@@ -458,6 +460,7 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_retention = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1170,7 +1173,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();
@@ -1199,7 +1203,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1225,22 +1229,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_retention;
SpinLockAcquire(&w->relmutex);
nonremovable_xid = w->oldest_nonremovable_xid;
+ stop_conflict_retention = w->stop_conflict_retention;
SpinLockRelease(&w->relmutex);
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that
+ * have stopped conflict retention.
+ */
+ if (stop_conflict_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;
}
@@ -1285,12 +1299,35 @@ 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, and update the sleep time before the next
* attempt.
*/
- if (retain_conflict_info)
+ else if (nretain_conflict_info)
{
bool updated = false;
@@ -1531,7 +1568,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;
@@ -1608,6 +1645,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_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 1a4c1ae624f..a748cb2eab4 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"
@@ -453,6 +454,7 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -4110,6 +4112,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_retention)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4279,6 +4285,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ bool stop_conflict_retention = false;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4340,22 +4348,61 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
- /*
- * 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);
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise,
+ * get the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
- elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
- LSN_FORMAT_ARGS(data->remote_lsn),
- XidFromFullTransactionId(data->candidate_xid));
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (!MySubscription->maxconflictretentionduration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ return;
+
+ stop_conflict_retention = true;
+ }
+
+ if (!stop_conflict_retention)
+ {
+ /*
+ * 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));
+ }
+ else
+ {
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ MyLogicalRepWorker->stop_conflict_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ MySubscription->maxconflictretentionduration));
+ }
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4419,6 +4466,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_retention = cause != RS_INVAL_NONE;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4590,6 +4682,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();
}
/*
@@ -5228,6 +5330,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 3a3884b3820..b74049b61b4 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -108,10 +108,11 @@ const char *const SlotInvalidationCauses[] = {
[RS_INVAL_WAL_REMOVED] = "wal_removed",
[RS_INVAL_HORIZON] = "rows_removed",
[RS_INVAL_WAL_LEVEL] = "wal_level_insufficient",
+ [RS_INVAL_CONFLICT_RETENTION_DURATION] = "conflict_retention_exceeds_max_duration",
};
/* Maximum number of invalidation causes */
-#define RS_INVAL_MAX_CAUSES RS_INVAL_WAL_LEVEL
+#define RS_INVAL_MAX_CAUSES RS_INVAL_CONFLICT_RETENTION_DURATION
StaticAssertDecl(lengthof(SlotInvalidationCauses) == (RS_INVAL_MAX_CAUSES + 1),
"array length mismatch");
@@ -1568,6 +1569,11 @@ ReportSlotInvalidation(ReplicationSlotInvalidationCause cause,
case RS_INVAL_WAL_LEVEL:
appendStringInfoString(&err_detail, _("Logical decoding on standby requires \"wal_level\" >= \"logical\" on the primary server."));
break;
+
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ appendStringInfo(&err_detail, _("The duration for retaining conflict information exceeds the maximum limit."));
+ break;
+
case RS_INVAL_NONE:
pg_unreachable();
}
@@ -1681,6 +1687,10 @@ InvalidatePossiblyObsoleteSlot(ReplicationSlotInvalidationCause cause,
if (SlotIsLogical(s))
invalidation_cause = cause;
break;
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ if (IsReservedSlotName(NameStr(s->data.name)))
+ invalidation_cause = cause;
+ break;
case RS_INVAL_NONE:
pg_unreachable();
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 7940dc5803f..677d3a332fe 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4868,6 +4868,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretainconflictinfo;
+ int i_submaxconflictretentionduration;
int i,
ntups;
@@ -4947,10 +4948,18 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 180000)
appendPQExpBufferStr(query,
- " s.subretainconflictinfo\n");
+ " s.subretainconflictinfo,\n");
else
appendPQExpBuffer(query,
- " false AS subretainconflictinfo\n");
+ " false AS subretainconflictinfo,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflictretentionduration\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflictretentionduration\n");
+
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4984,6 +4993,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo");
+ i_submaxconflictretentionduration = PQfnumber(res, "submaxconflictretentionduration");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5019,6 +5029,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretainconflictinfo =
(strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0);
+ subinfo[i].submaxconflictretentionduration =
+ atoi(PQgetvalue(res, i, i_submaxconflictretentionduration));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5280,6 +5292,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretainconflictinfo)
appendPQExpBufferStr(query, ", retain_conflict_info = true");
+ if (subinfo->submaxconflictretentionduration)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflictretentionduration);
+
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 0be12d05c93..f58716b5a74 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretainconflictinfo;
+ int submaxconflictretentionduration;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 54a89d5d62c..25571f4c68b 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6665,7 +6665,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, false, false};
if (pset.sversion < 100000)
{
@@ -6734,10 +6734,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 180000)
+ {
appendPQExpBuffer(&buf,
", subretainconflictinfo AS \"%s\"\n",
gettext_noop("Retain conflict info"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflictretentionduration AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index e0be3c6fd63..d899428e5d5 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,10 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin", "password_required",
+ "retain_conflict_info", "run_as_owner", "slot_name",
+ "streaming", "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,10 +3689,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin", "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_proc.dat b/src/include/catalog/pg_proc.dat
index 18560755d26..98ee6513f91 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5608,9 +5608,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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 0ac7c0b120c..f5677b53fad 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,12 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretainconflictinfo; /* True if information useful for
* conflict detection is retained */
+ int32 submaxconflictretentionduration; /* The maximum duration
+ * (in milliseconds) for
+ * which conflict
+ * information can be
+ * retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +142,10 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retainconflictinfo; /* True if information useful for conflict
* detection is retained */
+ int32 maxconflictretentionduration; /* The maximum duration (in
+ * milliseconds) for which
+ * conflict information can be
+ * 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/include/replication/slot.h b/src/include/replication/slot.h
index ea0485efa2e..70be49d97f6 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -63,6 +63,8 @@ typedef enum ReplicationSlotInvalidationCause
RS_INVAL_HORIZON,
/* wal_level insufficient for slot */
RS_INVAL_WAL_LEVEL,
+ /* duration of conflict info retention exceeds the maximum limit */
+ RS_INVAL_CONFLICT_RETENTION_DURATION,
} ReplicationSlotInvalidationCause;
extern PGDLLIMPORT const char *const SlotInvalidationCauses[];
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b09486017f4..40469f2df28 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_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 856a8349c50..12b92e3826c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2138,9 +2138,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,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index bff4cc051db..577493fe17b 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -434,19 +434,44 @@ WARNING: information for detecting conflicts cannot be fully retained when "tra
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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | 0 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 c65397e5ac6..17dd6aa26a9 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -303,6 +303,22 @@ ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.31.1
v25-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v25-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 73d20886f01ff504dd87467161654c0b826dbf9b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v25 5/6] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 163 ++++++++++++++++++
2 files changed, 164 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..75539b2cba9
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,163 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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;");
+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;");
+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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.31.1
v25-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v25-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From 8fca37b7858ec47f5e17931bc2f36d28272256fc Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v25 6/6] 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 05790396755..f1891e9f21f 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1626,6 +1626,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index c1c61e2ed6c..12f21c521f8 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 523bca04957..e8bcc30c9ad 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1379,6 +1379,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 3985e84d3a6..ecb81c202c2 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 a748cb2eab4..83e845c6e6e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2746,6 +2746,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2762,15 +2765,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2781,7 +2780,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2800,19 +2799,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3133,9 +3141,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3144,20 +3152,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3166,7 +3183,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3177,7 +3194,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 0f5e0a9778d..1b6436324a2 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2131,7 +2131,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2153,17 +2153,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 98ee6513f91..f938ccbcd0a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 c7db6defd3e..8af7ce4ce38 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"
@@ -671,7 +672,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 37454dc9513..91224b6ea60 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 12b92e3826c..5d05584816d 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,6 +2147,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,
@@ -2154,7 +2155,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 75539b2cba9..ec446191e95 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -120,6 +125,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -127,6 +139,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.31.1
On Wednesday, January 22, 2025 7:54 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Saturday, January 18, 2025 11:45 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I think invalidating the slot is OK and we could also let the apply
worker to automatic recovery as suggested in [1].Here is the V24 patch set. I modified 0004 patch to implement the slot
Invalidation part. Since the automatic recovery could be an
optimization and the discussion is in progress, I didn't implement that part.The implementation is in progress and I will include it in next version.
Here is the V25 patch set that includes the following change:
0001
* Per off-list discussion with Amit, I added few comments to mention the
reason of skipping advancing xid when table sync is in progress and to mention
that the advancement will not be delayed if changes are filtered out on
publisher via row/table filter.0004
* Fixed a bug that the launcher would advance the slot.xmin when some apply
workers have not yet started.* Fixed a bug that the launcher did not advance the slot.xmin even if one of the
apply worker has stopped conflict retention due to the lag.* Add a retain_conflict_info column in the pg_stat_subscription view to
indicate 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. Thanks Kuroda-san for contributing codes
off-list.
Here is V25 patch set which includes the following changes:
0004
* Addressed Nisha's comments[1]/messages/by-id/CABdArM69ukHVQMmnyKpwbYH8aLhXu-d4fGbSiTkq7BBgRWe0TA@mail.gmail.com.
* Fixed a cfbot failure[2]https://cirrus-ci.com/task/5006366856708096?logs=docs_build#L299 in the doc.
New 0005
* 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.
[1]: /messages/by-id/CABdArM69ukHVQMmnyKpwbYH8aLhXu-d4fGbSiTkq7BBgRWe0TA@mail.gmail.com
[2]: https://cirrus-ci.com/task/5006366856708096?logs=docs_build#L299
Best Regards,
Hou zj
Attachments:
v26-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v26-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From a139c85594b91247307fed3d5eab812d745f43ab Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v26 6/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 163 ++++++++++++++++++
2 files changed, 164 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..75539b2cba9
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,163 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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;");
+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;");
+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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
v26-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v26-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 8ba5aa5eaf7615f542d56b477251c356609d9beb Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 8 Jan 2025 15:48:14 +0800
Subject: [PATCH v26 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 90c23da5241..25623168e6f 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1626,6 +1626,21 @@ test_sub=# SELECT * FROM t1 ORDER BY id;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 6433a82f4e4..9839c9e80c1 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 523bca04957..e8bcc30c9ad 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1379,6 +1379,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 3985e84d3a6..ecb81c202c2 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);
/*
* Returns the fixed strategy number, if any, of the equality operator for the
@@ -253,7 +256,7 @@ retry:
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;
}
@@ -309,10 +312,13 @@ retry:
/*
* 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;
@@ -337,6 +343,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
@@ -412,7 +426,7 @@ retry:
/* 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;
@@ -463,6 +477,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 a236dbdf3bc..e4383dc8bd7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2745,6 +2745,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2761,15 +2764,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2780,7 +2779,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2799,19 +2798,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3132,9 +3140,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3143,20 +3151,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3165,7 +3182,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3176,7 +3193,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 0f5e0a9778d..1b6436324a2 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2131,7 +2131,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2153,17 +2153,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 98ee6513f91..f938ccbcd0a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5600,9 +5600,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 c7db6defd3e..8af7ce4ce38 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"
@@ -671,7 +672,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 37454dc9513..91224b6ea60 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 12b92e3826c..5d05584816d 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2147,6 +2147,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,
@@ -2154,7 +2155,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 75539b2cba9..ec446191e95 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -120,6 +125,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -127,6 +139,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
v26-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v26-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 41ce63173895675aa3970be4c22aaa15672dfede Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v26 1/4] 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 | 498 ++++++++++++++++++++-
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, 752 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index ab2f4a8a92f..3973b085f9c 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1197,7 +1197,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.
@@ -2302,7 +2306,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),
@@ -2322,7 +2326,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2335,8 +2339,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
@@ -2385,7 +2399,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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 334bf3e7aff..393a514ed2d 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,
@@ -3573,6 +3648,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
@@ -3651,6 +3727,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);
@@ -3677,6 +3755,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3772,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);
@@ -3706,6 +3810,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)
{
/*
@@ -3740,6 +3849,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,
@@ -3803,6 +3919,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
@@ -3838,7 +3956,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;
@@ -3916,6 +4033,385 @@ 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;
+ uint32 epoch;
+ TimestampTz now;
+
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest 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;
+
+ /*
+ * Compute the epoch of the oldest running transaction ID based on the
+ * next full transaction ID and its epoch.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId();
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, 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;
+ uint32 remote_epoch = data->remote_epoch;
+
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = FullTransactionIdFromEpochAndXid(data->remote_epoch,
+ data->remote_nextxid);
+
+ /* Compute the epoch of the remote oldest running transaction ID */
+ if (data->remote_oldestxid > data->remote_nextxid)
+ remote_epoch--;
+
+ remote_full_xid = FullTransactionIdFromEpochAndXid(remote_epoch,
+ 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 a0782b1bbf6..1299ed1eb5d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 d5aa5c295ae..58cfa2de17e 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2479,6 +2479,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v26-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v26-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From e4769d038ea03a301706c0c052739e8ba102cf51 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v26 2/4] 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 | 239 +++++++++++++++++-
.../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, 304 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a782f109982..cfdb5b5e102 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4736,6 +4736,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 47370e581ae..feb026d0c5a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29213,7 +29213,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29258,7 +29260,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29288,6 +29292,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29309,7 +29315,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index cf2b007806f..70d6fd62c97 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4740,7 +4740,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..e9869664d32 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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 +1189,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 +1271,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1325,150 @@ 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);
+ 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 bool
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
+{
+ TransactionId next_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_xmin;
+ uint32 xmin_epoch;
+
+ Assert(MyReplicationSlot);
+ Assert(FullTransactionIdIsValid(new_xmin));
+
+ /*
+ * Compute the epoch of the xmin value for the replication slot based on
+ * the next full transaction ID and its epoch.
+ */
+ next_full_xid = ReadNextFullTransactionId();
+ next_xid = XidFromFullTransactionId(next_full_xid);
+ xmin_epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the current
+ * xmin of the replication slot. This handles the case where transaction
+ * ID wraparound has occurred.
+ */
+ if (next_xid < MyReplicationSlot->data.xmin)
+ xmin_epoch--;
+
+ full_xmin = FullTransactionIdFromEpochAndXid(xmin_epoch,
+ MyReplicationSlot->data.xmin);
+
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 79b60df7cf0..64b43346a90 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4665,7 +4665,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 393a514ed2d..42a53ee8a56 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4353,6 +4353,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 b30e0473e1c..3a3884b3820 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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 82b202f3305..7b29f1814db 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 bf62b36ad07..ea0485efa2e 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.
*
@@ -261,7 +268,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);
--
2.30.0.windows.2
v26-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v26-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 325521fb785bad5e0b1f7cd229a29d1663bb6f72 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:15:07 +0800
Subject: [PATCH v26 3/4] 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 | 53 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 442 insertions(+), 127 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index d3036c5ba9d..14c094e1b5e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8051,6 +8051,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index ab683cf111e..90c23da5241 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2195,7 +2195,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 46868bf7e89..c87d8a84d7c 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1367,7 +1367,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 e9869664d32..850dd0aaebe 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1190,16 +1192,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);
@@ -1209,10 +1221,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;
@@ -1238,7 +1251,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,
@@ -1276,7 +1290,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1290,12 +1304,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1326,7 +1342,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
@@ -1368,7 +1384,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1379,7 +1395,10 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
uint32 xmin_epoch;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+ Assert(TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
/*
* Compute the epoch of the xmin value for the replication slot based on
@@ -1457,8 +1476,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 42a53ee8a56..41d6e1401a4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4098,6 +4098,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
@@ -4514,6 +4518,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 ||
@@ -4523,7 +4536,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 8f73a5df956..7940dc5803f 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4867,6 +4867,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4939,11 +4940,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4976,6 +4983,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");
@@ -5009,6 +5017,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))
@@ -5267,6 +5277,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 f62b564ed1b..0be12d05c93 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -680,6 +680,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index 8c0ad8439eb..63018181c11 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6676,7 +6676,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)
{
@@ -6744,6 +6744,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v26-0004-add-a-max_conflict_retention_duration-subscripti.patchapplication/octet-stream; name=v26-0004-add-a-max_conflict_retention_duration-subscripti.patchDownload
From a31b6721c0e5c662137e37fffd45e6587756cc58 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 15 Jan 2025 16:46:15 +0800
Subject: [PATCH v26 4/4] add a max_conflict_retention_duration subscription
option
This commit introduces the max_conflict_retention_duration subscription option,
designed to prevent excessive accumulation of dead tuples when
retain_conflict_info is enabled 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.
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/catalogs.sgml | 10 ++
doc/src/sgml/monitoring.sgml | 13 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 40 +++++
doc/src/sgml/system-views.sgml | 11 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/launcher.c | 79 ++++++++-
src/backend/replication/logical/worker.c | 133 ++++++++++++--
src/backend/replication/slot.c | 13 +-
src/bin/pg_dump/pg_dump.c | 19 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 16 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 10 ++
src/include/replication/slot.h | 2 +
src/include/replication/worker_internal.h | 6 +
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 193 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
22 files changed, 494 insertions(+), 132 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 14c094e1b5e..3682a6c3137 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8063,6 +8063,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflictretentionduration</structfield> <type>interger</type>
+ </para>
+ <para>
+ If not zero, the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5888fae2b5..6433a82f4e4 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2101,6 +2101,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e5415c3150d..b772cefe075 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 61374b74d1c..fef6c0757e3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -466,6 +466,46 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Specifies the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 8e2b0a7927b..cce6214df50 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -2620,6 +2620,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
perform logical decoding. It is set only for logical slots.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 39cfae43d6f..d3d9cc1779c 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retainconflictinfo = subform->subretainconflictinfo;
+ sub->maxconflictretentionduration = subform->submaxconflictretentionduration;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c87d8a84d7c..523bca04957 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -976,7 +976,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);
@@ -1367,8 +1368,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretainconflictinfo, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretainconflictinfo, submaxconflictretentionduration, 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 51bd52e48a3..13199740320 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retainconflictinfo;
+ int32 maxconflictretentionduration;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO))
opts->retainconflictinfo = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflictretentionduration = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO;
opts->retainconflictinfo = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflictretentionduration = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -695,6 +709,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretainconflictinfo - 1] =
BoolGetDatum(opts.retainconflictinfo);
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1193,7 +1209,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1376,6 +1393,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_conflict_info = opts.retainconflictinfo;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
+ replaces[Anum_pg_subscription_submaxconflictretentionduration - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 850dd0aaebe..11dd5edccae 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"
@@ -42,6 +43,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
@@ -458,6 +460,7 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_retention = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1170,7 +1173,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();
@@ -1199,7 +1203,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1225,22 +1229,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_retention;
SpinLockAcquire(&w->relmutex);
nonremovable_xid = w->oldest_nonremovable_xid;
+ stop_conflict_retention = w->stop_conflict_retention;
SpinLockRelease(&w->relmutex);
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers
+ * that have stopped conflict retention.
+ */
+ if (stop_conflict_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;
}
@@ -1285,12 +1299,35 @@ 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, and update the sleep time before the next
* attempt.
*/
- if (retain_conflict_info)
+ else if (nretain_conflict_info)
{
bool updated = false;
@@ -1531,7 +1568,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;
@@ -1608,6 +1645,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_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 41d6e1401a4..57e8497f57d 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"
@@ -453,6 +454,7 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -3853,7 +3855,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_retention &&
+ data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval)
wait_time = Min(wait_time, data.xid_advance_interval);
rc = WaitLatchOrSocket(MyLatch,
@@ -4110,6 +4113,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_retention)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4279,6 +4286,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ bool stop_conflict_retention = false;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4340,22 +4349,61 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
- /*
- * 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);
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise,
+ * get the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
- elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
- LSN_FORMAT_ARGS(data->remote_lsn),
- XidFromFullTransactionId(data->candidate_xid));
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (!MySubscription->maxconflictretentionduration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ return;
+
+ stop_conflict_retention = true;
+ }
+
+ if (!stop_conflict_retention)
+ {
+ /*
+ * 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));
+ }
+ else
+ {
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ MyLogicalRepWorker->stop_conflict_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ MySubscription->maxconflictretentionduration));
+ }
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4419,6 +4467,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_retention = cause != RS_INVAL_NONE;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4590,6 +4683,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();
}
/*
@@ -5228,6 +5331,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 3a3884b3820..d6b04b66f90 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -108,10 +108,11 @@ const char *const SlotInvalidationCauses[] = {
[RS_INVAL_WAL_REMOVED] = "wal_removed",
[RS_INVAL_HORIZON] = "rows_removed",
[RS_INVAL_WAL_LEVEL] = "wal_level_insufficient",
+ [RS_INVAL_CONFLICT_RETENTION_DURATION] = "conflict_retention_exceeds_max_duration",
};
/* Maximum number of invalidation causes */
-#define RS_INVAL_MAX_CAUSES RS_INVAL_WAL_LEVEL
+#define RS_INVAL_MAX_CAUSES RS_INVAL_CONFLICT_RETENTION_DURATION
StaticAssertDecl(lengthof(SlotInvalidationCauses) == (RS_INVAL_MAX_CAUSES + 1),
"array length mismatch");
@@ -1568,6 +1569,11 @@ ReportSlotInvalidation(ReplicationSlotInvalidationCause cause,
case RS_INVAL_WAL_LEVEL:
appendStringInfoString(&err_detail, _("Logical decoding on standby requires \"wal_level\" >= \"logical\" on the primary server."));
break;
+
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ appendStringInfo(&err_detail, _("The duration for retaining conflict information exceeds the maximum limit defined by the subscription's max_conflict_retention_duration setting"));
+ break;
+
case RS_INVAL_NONE:
pg_unreachable();
}
@@ -1681,6 +1687,10 @@ InvalidatePossiblyObsoleteSlot(ReplicationSlotInvalidationCause cause,
if (SlotIsLogical(s))
invalidation_cause = cause;
break;
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ if (IsReservedSlotName(NameStr(s->data.name)))
+ invalidation_cause = cause;
+ break;
case RS_INVAL_NONE:
pg_unreachable();
}
@@ -1834,6 +1844,7 @@ InvalidatePossiblyObsoleteSlot(ReplicationSlotInvalidationCause cause,
* - RS_INVAL_HORIZON: requires a snapshot <= the given horizon in the given
* db; dboid may be InvalidOid for shared relations
* - RS_INVAL_WAL_LEVEL: is logical
+ * - RS_INVAL_CONFLICT_RETENTION_DURATION: is "pg_conflict_detection"
*
* NB - this runs as part of checkpoint, so avoid raising errors if possible.
*/
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 7940dc5803f..677d3a332fe 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4868,6 +4868,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretainconflictinfo;
+ int i_submaxconflictretentionduration;
int i,
ntups;
@@ -4947,10 +4948,18 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 180000)
appendPQExpBufferStr(query,
- " s.subretainconflictinfo\n");
+ " s.subretainconflictinfo,\n");
else
appendPQExpBuffer(query,
- " false AS subretainconflictinfo\n");
+ " false AS subretainconflictinfo,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflictretentionduration\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflictretentionduration\n");
+
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4984,6 +4993,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo");
+ i_submaxconflictretentionduration = PQfnumber(res, "submaxconflictretentionduration");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5019,6 +5029,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretainconflictinfo =
(strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0);
+ subinfo[i].submaxconflictretentionduration =
+ atoi(PQgetvalue(res, i, i_submaxconflictretentionduration));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5280,6 +5292,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretainconflictinfo)
appendPQExpBufferStr(query, ", retain_conflict_info = true");
+ if (subinfo->submaxconflictretentionduration)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflictretentionduration);
+
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 0be12d05c93..f58716b5a74 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretainconflictinfo;
+ int submaxconflictretentionduration;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 63018181c11..e855513bdbd 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6676,7 +6676,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, false, false};
if (pset.sversion < 100000)
{
@@ -6745,10 +6745,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 180000)
+ {
appendPQExpBuffer(&buf,
", subretainconflictinfo AS \"%s\"\n",
gettext_noop("Retain conflict info"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflictretentionduration AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index e0be3c6fd63..d899428e5d5 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,10 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin", "password_required",
+ "retain_conflict_info", "run_as_owner", "slot_name",
+ "streaming", "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,10 +3689,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin", "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_proc.dat b/src/include/catalog/pg_proc.dat
index 18560755d26..98ee6513f91 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5608,9 +5608,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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 0ac7c0b120c..f5677b53fad 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,12 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretainconflictinfo; /* True if information useful for
* conflict detection is retained */
+ int32 submaxconflictretentionduration; /* The maximum duration
+ * (in milliseconds) for
+ * which conflict
+ * information can be
+ * retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +142,10 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retainconflictinfo; /* True if information useful for conflict
* detection is retained */
+ int32 maxconflictretentionduration; /* The maximum duration (in
+ * milliseconds) for which
+ * conflict information can be
+ * 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/include/replication/slot.h b/src/include/replication/slot.h
index ea0485efa2e..70be49d97f6 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -63,6 +63,8 @@ typedef enum ReplicationSlotInvalidationCause
RS_INVAL_HORIZON,
/* wal_level insufficient for slot */
RS_INVAL_WAL_LEVEL,
+ /* duration of conflict info retention exceeds the maximum limit */
+ RS_INVAL_CONFLICT_RETENTION_DURATION,
} ReplicationSlotInvalidationCause;
extern PGDLLIMPORT const char *const SlotInvalidationCauses[];
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b09486017f4..40469f2df28 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_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 856a8349c50..12b92e3826c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2138,9 +2138,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,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index bff4cc051db..577493fe17b 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -434,19 +434,44 @@ WARNING: information for detecting conflicts cannot be fully retained when "tra
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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | 0 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 c65397e5ac6..17dd6aa26a9 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -303,6 +303,22 @@ ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.30.0.windows.2
v26-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v26-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 536f907bb695eb3c6a8adede675cce5e4c042bd5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 23 Jan 2025 13:29:37 +0800
Subject: [PATCH v26 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/ref/create_subscription.sgml | 5 +-
src/backend/replication/logical/launcher.c | 37 +++---
src/backend/replication/logical/worker.c | 135 ++++++++-------------
3 files changed, 73 insertions(+), 104 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index c6d87255a39..b31c89ec693 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -488,7 +488,10 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 11dd5edccae..923680f53c6 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -460,7 +460,8 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
- worker->stop_conflict_retention = false;
+ worker->stop_conflict_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1299,27 +1300,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1331,6 +1325,17 @@ ApplyLauncherMain(Datum main_arg)
{
bool updated = false;
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
updated = advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 57e8497f57d..a236dbdf3bc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -454,7 +454,6 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -4113,10 +4112,6 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
if (!am_leader_apply_worker())
return;
- /* Exit early if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_retention)
- return;
-
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4286,7 +4281,9 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ TimestampTz now;
bool stop_conflict_retention = false;
+ bool status_changed;
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4349,30 +4346,35 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Check if changes up to the remote_lsn have been applied and flushed */
- if (last_flushpos < data->remote_lsn)
- {
- TimestampTz now;
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
- /*
- * Use last_recv_time when applying changes in the loop; otherwise,
- * get the latest timestamp.
- */
- now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
+ /*
+ * If the wait time has exceeded the maximum limit
+ * (max_conflict_retention_duration), stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (MySubscription->maxconflictretentionduration &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ stop_conflict_retention = true;
- /*
- * If the wait time has not exceeded the maximum limit
- * (max_conflict_retention_duration), continue waiting for the changes
- * to be applied. Otherwise, stop tracking the non-removable
- * transaction ID by this apply worker.
- */
- if (!MySubscription->maxconflictretentionduration ||
- !TimestampDifferenceExceeds(data->candidate_xid_time, now,
- MySubscription->maxconflictretentionduration))
- return;
+ /*
+ * Return if the apply worker is retaining conflict information, and the
+ * changes up to the remote_lsn are not yet applied and flushed.
+ */
+ if (!stop_conflict_retention && last_flushpos < data->remote_lsn)
+ return;
- stop_conflict_retention = true;
- }
+ /*
+ * Determine whether the apply worker is about to stop or resume conflict
+ * retention.
+ */
+ if (MyLogicalRepWorker->stop_conflict_retention != stop_conflict_retention)
+ status_changed = true;
if (!stop_conflict_retention)
{
@@ -4384,13 +4386,25 @@ wait_for_local_flush(RetainConflictInfoData *data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ if (status_changed)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->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),
+ MySubscription->maxconflictretentionduration));
+
elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(data->remote_lsn),
XidFromFullTransactionId(data->candidate_xid));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
}
- else
+ else if (status_changed)
{
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
@@ -4403,10 +4417,14 @@ wait_for_local_flush(RetainConflictInfoData *data)
errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
LSN_FORMAT_ARGS(data->remote_lsn),
MySubscription->maxconflictretentionduration));
- }
- /* Notify launcher to update the xmin of the conflict slot */
- ApplyLauncherWakeup();
+ /*
+ * Notify launcher to either update the xmin of the conflict slot or
+ * invalidate the slot if no other workers are retaining conflict
+ * information.
+ */
+ ApplyLauncherWakeup();
+ }
/*
* Reset all data fields except those used to determine the timing for the
@@ -4467,51 +4485,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_retention = cause != RS_INVAL_NONE;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-}
-
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4683,16 +4656,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();
}
/*
@@ -5331,8 +5294,6 @@ InitializeLogRepWorker(void)
MySubscription->name)));
CommitTransactionCommand();
-
- update_conflict_retention_status();
}
/*
--
2.30.0.windows.2
On Thu, Jan 23, 2025 at 3:47 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wednesday, January 22, 2025 7:54 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Saturday, January 18, 2025 11:45 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I think invalidating the slot is OK and we could also let the apply
worker to automatic recovery as suggested in [1].Here is the V24 patch set. I modified 0004 patch to implement the slot
Invalidation part. Since the automatic recovery could be an
optimization and the discussion is in progress, I didn't implement that part.The implementation is in progress and I will include it in next version.
Here is the V25 patch set that includes the following change:
0001
* Per off-list discussion with Amit, I added few comments to mention the
reason of skipping advancing xid when table sync is in progress and to mention
that the advancement will not be delayed if changes are filtered out on
publisher via row/table filter.0004
* Fixed a bug that the launcher would advance the slot.xmin when some apply
workers have not yet started.* Fixed a bug that the launcher did not advance the slot.xmin even if one of the
apply worker has stopped conflict retention due to the lag.* Add a retain_conflict_info column in the pg_stat_subscription view to
indicate 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. Thanks Kuroda-san for contributing codes
off-list.Here is V25 patch set which includes the following changes:
0004
* Addressed Nisha's comments[1].
* Fixed a cfbot failure[2] in the doc.
I have one question about the 0004 patch; it implemented
max_conflict_retntion_duration as a subscription parameter. But the
launcher invalidates the pg_conflict_detection slot only if all
subscriptions with retain_conflict_info stopped retaining dead tuples
due to the max_conflict_retention_duration parameter. Therefore, even
if users set the parameter to a low value to avoid table bloats, it
would not make sense if other subscriptions set it to a larger value.
Is my understanding correct?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Fri, Jan 31, 2025 at 4:10 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I have one question about the 0004 patch; it implemented
max_conflict_retntion_duration as a subscription parameter. But the
launcher invalidates the pg_conflict_detection slot only if all
subscriptions with retain_conflict_info stopped retaining dead tuples
due to the max_conflict_retention_duration parameter. Therefore, even
if users set the parameter to a low value to avoid table bloats, it
would not make sense if other subscriptions set it to a larger value.
Is my understanding correct?
Yes, your understanding is correct. I think this could be helpful
during resolution because the worker for which the duration has
exceeded cannot detect conflicts reliably but others can. So, this
info can be useful while performing resolutions. Do you have an
opinion/suggestion on this matter?
--
With Regards,
Amit Kapila.
On Thu, Jan 30, 2025 at 10:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jan 31, 2025 at 4:10 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I have one question about the 0004 patch; it implemented
max_conflict_retntion_duration as a subscription parameter. But the
launcher invalidates the pg_conflict_detection slot only if all
subscriptions with retain_conflict_info stopped retaining dead tuples
due to the max_conflict_retention_duration parameter. Therefore, even
if users set the parameter to a low value to avoid table bloats, it
would not make sense if other subscriptions set it to a larger value.
Is my understanding correct?Yes, your understanding is correct. I think this could be helpful
during resolution because the worker for which the duration has
exceeded cannot detect conflicts reliably but others can. So, this
info can be useful while performing resolutions. Do you have an
opinion/suggestion on this matter?
I imagined a scenario like where two apply workers are running and
have different max_conflict_retention_duration values (say '5 min' and
'15 min'). Suppose both workers are roughly the same behind the
publisher(s), when both workers cannot advance the workers' xmin
values for 5 min or longer, one worker stops retaining dead tuples.
However, the pg_conflict_detection slot is not invalidated yet since
another worker is still using it, so both workers would continue to be
getting slower. The subscriber would end up retaining dead tuples
until both workers are behind for 15 min or longer, before
invalidating the slot. In this case, stopping dead tuple retention on
the first worker would help neither advance the slot's xmin nor
improve another worker's performance. I was not sure of the point of
making the max_conflict_retention_duration a per-subscription
parameter.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Sat, Feb 1, 2025 at 2:54 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Jan 30, 2025 at 10:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jan 31, 2025 at 4:10 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I have one question about the 0004 patch; it implemented
max_conflict_retntion_duration as a subscription parameter. But the
launcher invalidates the pg_conflict_detection slot only if all
subscriptions with retain_conflict_info stopped retaining dead tuples
due to the max_conflict_retention_duration parameter. Therefore, even
if users set the parameter to a low value to avoid table bloats, it
would not make sense if other subscriptions set it to a larger value.
Is my understanding correct?Yes, your understanding is correct. I think this could be helpful
during resolution because the worker for which the duration has
exceeded cannot detect conflicts reliably but others can. So, this
info can be useful while performing resolutions. Do you have an
opinion/suggestion on this matter?I imagined a scenario like where two apply workers are running and
have different max_conflict_retention_duration values (say '5 min' and
'15 min'). Suppose both workers are roughly the same behind the
publisher(s), when both workers cannot advance the workers' xmin
values for 5 min or longer, one worker stops retaining dead tuples.
However, the pg_conflict_detection slot is not invalidated yet since
another worker is still using it, so both workers would continue to be
getting slower. The subscriber would end up retaining dead tuples
until both workers are behind for 15 min or longer, before
invalidating the slot. In this case, stopping dead tuple retention on
the first worker would help neither advance the slot's xmin nor
improve another worker's performance.
Won't the same be true for 'retain_conflict_info' option as well? I
mean even if one worker is retaining dead tuples, the performance of
others will also be impacted.
I was not sure of the point of
making the max_conflict_retention_duration a per-subscription
parameter.
The idea is to keep it at the same level as the other related
parameter 'retain_conflict_info'. It could be useful for cases where
publishers are from two different nodes (NP1 and NP2) and we have
separate subscriptions for both nodes. Now, it is possible that users
won't expect conflicts on the tables from one of the nodes NP1 then
she could choose to enable 'retain_conflict_info' and
'max_conflict_retention_duration' only for the subscription pointing
to publisher NP2.
Now, say the publisher node that can generate conflicts (NP2) has
fewer writes and the corresponding apply worker could easily catch up
and almost always be in sync with the publisher. In contrast, the
other node that has no conflicts has a large number of writes. In such
cases, giving new options at the subscription level will be helpful.
If we want to provide it at the global level, then the performance or
dead tuple control may not be any better than the current patch but
won't allow the provision for the above kinds of cases. Second, adding
two new GUCs is another thing I want to prevent. But OTOH, the
implementation could be slightly simpler if we provide these options
as GUC though I am not completely sure of that point. Having said
that, I am open to changing it to a non-subscription level. Do you
think it would be better to provide one or both of these parameters as
GUCs or do you have something else in mind?
--
With Regards,
Amit Kapila.
On Fri, Jan 31, 2025 at 9:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Feb 1, 2025 at 2:54 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Jan 30, 2025 at 10:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jan 31, 2025 at 4:10 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I have one question about the 0004 patch; it implemented
max_conflict_retntion_duration as a subscription parameter. But the
launcher invalidates the pg_conflict_detection slot only if all
subscriptions with retain_conflict_info stopped retaining dead tuples
due to the max_conflict_retention_duration parameter. Therefore, even
if users set the parameter to a low value to avoid table bloats, it
would not make sense if other subscriptions set it to a larger value.
Is my understanding correct?Yes, your understanding is correct. I think this could be helpful
during resolution because the worker for which the duration has
exceeded cannot detect conflicts reliably but others can. So, this
info can be useful while performing resolutions. Do you have an
opinion/suggestion on this matter?I imagined a scenario like where two apply workers are running and
have different max_conflict_retention_duration values (say '5 min' and
'15 min'). Suppose both workers are roughly the same behind the
publisher(s), when both workers cannot advance the workers' xmin
values for 5 min or longer, one worker stops retaining dead tuples.
However, the pg_conflict_detection slot is not invalidated yet since
another worker is still using it, so both workers would continue to be
getting slower. The subscriber would end up retaining dead tuples
until both workers are behind for 15 min or longer, before
invalidating the slot. In this case, stopping dead tuple retention on
the first worker would help neither advance the slot's xmin nor
improve another worker's performance.Won't the same be true for 'retain_conflict_info' option as well? I
mean even if one worker is retaining dead tuples, the performance of
others will also be impacted.
I guess the situation might be a bit different. It's a user's choice
to disable retain_conflict_info, and it should be done manually. That
is, in this case, I think users will be able to figure out that both
apply workers are the same behind the publishers and they need to
disable retain_conflict_info on both subscriptions in order to remove
accumulated dead tuples (which is the cause of performance dip).
On the other hand, ISTM max_conflict_retentation_duration is something
like a switch to recover the system performance by automatically
disabling retain_conflict_info (and it will automatically go back to
be enabled again). I guess users who use the
max_conflict_retention_duration would expect that the system
performance will tend to recover by automatically disabling
reatin_conflict_info if the apply worker is lagging for longer than
the specified value. However, there are cases where this cannot be
expected.
I was not sure of the point of
making the max_conflict_retention_duration a per-subscription
parameter.The idea is to keep it at the same level as the other related
parameter 'retain_conflict_info'. It could be useful for cases where
publishers are from two different nodes (NP1 and NP2) and we have
separate subscriptions for both nodes. Now, it is possible that users
won't expect conflicts on the tables from one of the nodes NP1 then
she could choose to enable 'retain_conflict_info' and
'max_conflict_retention_duration' only for the subscription pointing
to publisher NP2.Now, say the publisher node that can generate conflicts (NP2) has
fewer writes and the corresponding apply worker could easily catch up
and almost always be in sync with the publisher. In contrast, the
other node that has no conflicts has a large number of writes. In such
cases, giving new options at the subscription level will be helpful.If we want to provide it at the global level, then the performance or
dead tuple control may not be any better than the current patch but
won't allow the provision for the above kinds of cases. Second, adding
two new GUCs is another thing I want to prevent. But OTOH, the
implementation could be slightly simpler if we provide these options
as GUC though I am not completely sure of that point. Having said
that, I am open to changing it to a non-subscription level. Do you
think it would be better to provide one or both of these parameters as
GUCs or do you have something else in mind?
It makes sense to me to have the retain_conflict_info as a
subscription-level parameter. I was thinking of making only
max_conflict_retention_duration a global parameter, but I might be
missing something. With a subscription-level
max_conflict_retention_duration, how can users choose the setting
values for each subscription, and is there a case that can be covered
only by a subscription-level max_conflict_retention_duration?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Sat, Feb 1, 2025 at 10:37 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Feb 1, 2025 at 2:54 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Jan 30, 2025 at 10:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jan 31, 2025 at 4:10 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I have one question about the 0004 patch; it implemented
max_conflict_retntion_duration as a subscription parameter. But the
launcher invalidates the pg_conflict_detection slot only if all
subscriptions with retain_conflict_info stopped retaining dead tuples
due to the max_conflict_retention_duration parameter. Therefore, even
if users set the parameter to a low value to avoid table bloats, it
would not make sense if other subscriptions set it to a larger value.
Is my understanding correct?Yes, your understanding is correct. I think this could be helpful
during resolution because the worker for which the duration has
exceeded cannot detect conflicts reliably but others can. So, this
info can be useful while performing resolutions. Do you have an
opinion/suggestion on this matter?I imagined a scenario like where two apply workers are running and
have different max_conflict_retention_duration values (say '5 min' and
'15 min'). Suppose both workers are roughly the same behind the
publisher(s), when both workers cannot advance the workers' xmin
values for 5 min or longer, one worker stops retaining dead tuples.
However, the pg_conflict_detection slot is not invalidated yet since
another worker is still using it, so both workers would continue to be
getting slower. The subscriber would end up retaining dead tuples
until both workers are behind for 15 min or longer, before
invalidating the slot. In this case, stopping dead tuple retention on
the first worker would help neither advance the slot's xmin nor
improve another worker's performance.Won't the same be true for 'retain_conflict_info' option as well? I
mean even if one worker is retaining dead tuples, the performance of
others will also be impacted.
+1
I was not sure of the point of
making the max_conflict_retention_duration a per-subscription
parameter.The idea is to keep it at the same level as the other related
parameter 'retain_conflict_info'. It could be useful for cases where
publishers are from two different nodes (NP1 and NP2) and we have
separate subscriptions for both nodes. Now, it is possible that users
won't expect conflicts on the tables from one of the nodes NP1 then
she could choose to enable 'retain_conflict_info' and
'max_conflict_retention_duration' only for the subscription pointing
to publisher NP2.Now, say the publisher node that can generate conflicts (NP2) has
fewer writes and the corresponding apply worker could easily catch up
and almost always be in sync with the publisher. In contrast, the
other node that has no conflicts has a large number of writes. In such
cases, giving new options at the subscription level will be helpful.If we want to provide it at the global level, then the performance or
dead tuple control may not be any better than the current patch but
won't allow the provision for the above kinds of cases. Second, adding
two new GUCs is another thing I want to prevent. But OTOH, the
implementation could be slightly simpler if we provide these options
as GUC though I am not completely sure of that point. Having said
that, I am open to changing it to a non-subscription level. Do you
think it would be better to provide one or both of these parameters as
GUCs or do you have something else in mind?
I agree with this analogy. It seems that
'max_conflict_retention_duration' is quite similar to
'retain_conflict_info'. In both cases, the slot for retaining dead
tuples is shared among all subscribers. However, these subscribers may
be receiving data from different publishers and even different nodes.
Therefore, the decision on whether to wait and for how long should be
made at the subscriber level.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wed, Feb 5, 2025 at 6:00 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jan 31, 2025 at 9:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I was not sure of the point of
making the max_conflict_retention_duration a per-subscription
parameter.The idea is to keep it at the same level as the other related
parameter 'retain_conflict_info'. It could be useful for cases where
publishers are from two different nodes (NP1 and NP2) and we have
separate subscriptions for both nodes. Now, it is possible that users
won't expect conflicts on the tables from one of the nodes NP1 then
she could choose to enable 'retain_conflict_info' and
'max_conflict_retention_duration' only for the subscription pointing
to publisher NP2.Now, say the publisher node that can generate conflicts (NP2) has
fewer writes and the corresponding apply worker could easily catch up
and almost always be in sync with the publisher. In contrast, the
other node that has no conflicts has a large number of writes. In such
cases, giving new options at the subscription level will be helpful.If we want to provide it at the global level, then the performance or
dead tuple control may not be any better than the current patch but
won't allow the provision for the above kinds of cases. Second, adding
two new GUCs is another thing I want to prevent. But OTOH, the
implementation could be slightly simpler if we provide these options
as GUC though I am not completely sure of that point. Having said
that, I am open to changing it to a non-subscription level. Do you
think it would be better to provide one or both of these parameters as
GUCs or do you have something else in mind?It makes sense to me to have the retain_conflict_info as a
subscription-level parameter. I was thinking of making only
max_conflict_retention_duration a global parameter, but I might be
missing something. With a subscription-level
max_conflict_retention_duration, how can users choose the setting
values for each subscription, and is there a case that can be covered
only by a subscription-level max_conflict_retention_duration?
Users can configure depending on the workload of the publisher
considering the publishers are different nodes as explained in my
previous response. Also, I think it will help in resolutions where the
worker for which the duration for updating the worker_level xmin has
not exceeded the max_conflict_retention_duration can reliably detect
update_delete. Then this parameter will only be required for
subscriptions that have enabled retain_conflict_info. I am not
completely sure if these are reasons enough to keep at the
subscription level but OTOH Dilip also seems to favor keeping
max_conflict_retention_duration at susbcription-level.
--
With Regards,
Amit Kapila.
On Thu, Jan 23, 2025 at 5:17 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I was reviewing v26 patch set and have some comments so far I reviewed
0001 so most of the comments/question are from this patch.
comments on v26-0001
1.
+ next_full_xid = ReadNextFullTransactionId();
+ epoch = EpochFromFullTransactionId(next_full_xid);
+
+ /*
+ * Adjust the epoch if the next transaction ID is less than the oldest
+ * running transaction ID. This handles the case where transaction ID
+ * wraparound has occurred.
+ */
+ if (oldest_running_xid > XidFromFullTransactionId(next_full_xid))
+ epoch--;
+
+ full_xid = FullTransactionIdFromEpochAndXid(epoch, oldest_running_xid);
I think you can directly use the 'AdjustToFullTransactionId()'
function here, maybe we can move that somewhere else and make that
non-static function.
2.
+ /*
+ * 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."));
I don't fully understand the purpose of this check. Based on the
comments in RetainConflictInfoData, if I understand correctly,
candidate_xid_time represents the time when the candidate is
determined, and reply_time indicates the time of the reply from the
publisher. Why do we expect these two timestamps to have zero
difference to ensure clock synchronization?
3.
+ /*
+ * Use last_recv_time when applying changes in the loop; otherwise, get
+ * the latest timestamp.
+ */
+ now = data->last_recv_time ? data->last_recv_time : GetCurrentTimestamp();
Can you explain in the comment what's the logic behind using
last_recv_time here? Why not just compare 'candidate_xid_time' vs
current timestamp?
4.
Comment of v26-0004 doesn't clearly explain that once retention
stopped after reaching 'max_conflict_retention_duration' will it
resume back?
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Wednesday, February 5, 2025 3:45 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Thu, Jan 23, 2025 at 5:17 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:I was reviewing v26 patch set and have some comments so far I reviewed
0001 so most of the comments/question are from this patch.comments on v26-0001
Thanks for the comments !
1. + next_full_xid = ReadNextFullTransactionId(); epoch = + EpochFromFullTransactionId(next_full_xid); + + /* + * Adjust the epoch if the next transaction ID is less than the oldest + * running transaction ID. This handles the case where transaction ID + * wraparound has occurred. + */ + if (oldest_running_xid > XidFromFullTransactionId(next_full_xid)) + epoch--; + + full_xid = FullTransactionIdFromEpochAndXid(epoch, + oldest_running_xid);I think you can directly use the 'AdjustToFullTransactionId()'
function here, maybe we can move that somewhere else and make that
non-static function.
Thanks for the suggestion. I used the existing API
FullTransactionIdFromAllowableAt() in this version since it's also convenient
to use. We could expose AdjustToFullTransactionId as well, but I feel we could
do that separately.
2. + /* + * 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."));I don't fully understand the purpose of this check. Based on the comments in
RetainConflictInfoData, if I understand correctly, candidate_xid_time
represents the time when the candidate is determined, and reply_time
indicates the time of the reply from the publisher. Why do we expect these two
timestamps to have zero difference to ensure clock synchronization?
The check is to prevent the case when the publisher's clock is behind of the
subscriber's. Normally, the reply_time should be greater than the
candidate_xid_time if clock is in sync, but if the publisher's clock is behind,
then reply_time could be earlier than the candidate_xid_time, in which case
this ERROR would be triggered.
3. + /* + * Use last_recv_time when applying changes in the loop; otherwise, get + * the latest timestamp. + */ + now = data->last_recv_time ? data->last_recv_time : + GetCurrentTimestamp();Can you explain in the comment what's the logic behind using last_recv_time
here? Why not just compare 'candidate_xid_time' vs current timestamp?
The intention is to avoid unnecessary system time retrieval when we already
have the last_recv_time that can be directly used here (since it's fetched just
before invoking this function). It may not save a lot but It looks efficient to
reuse the existing timestamp when available. I added the same in comments.
4.
Comment of v26-0004 doesn't clearly explain that once retention stopped after
reaching 'max_conflict_retention_duration' will it resume back?
I Added some in commit message and comments of 0004.
Here is the V27 patch set which addressed comment above.
Best Regards,
Hou zj
Attachments:
v27-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v27-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 536d86427fc23230964a4f21ccd7c77ec0f2c822 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v27 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 defined by wal_receiver_status_interval
to minimize performance impact.
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 fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..41fb4fc5025 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 d331ab90d78..e3753d5d21f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 6966037d2ef..409b0de4eba 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,
@@ -3573,6 +3648,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
@@ -3651,6 +3727,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);
@@ -3677,6 +3755,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3692,8 +3772,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);
@@ -3706,6 +3810,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)
{
/*
@@ -3740,6 +3849,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,
@@ -3803,6 +3919,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
@@ -3838,7 +3956,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;
@@ -3916,6 +4033,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 446d10c1a7d..9d27121a2e4 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2313,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2659,6 +2665,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 9a3bee93dec..345b006248e 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2480,6 +2480,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v27-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v27-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 3f461ec6d44ad32030796e63afb4b58a1948703b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v27 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 | 227 +++++++++++++++++-
.../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, 292 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index a782f109982..cfdb5b5e102 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4736,6 +4736,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 7efc81936ab..5c0891f07f1 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29264,7 +29264,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29309,7 +29311,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29339,6 +29343,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29360,7 +29366,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 473de6710d7..5fed28c0f29 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4752,7 +4752,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..2ef6594b5ce 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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 +1189,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 +1271,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1325,138 @@ 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 bool
+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 false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 10a37667a51..857a0170333 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4664,7 +4664,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 409b0de4eba..19002b52814 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4346,6 +4346,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 fe5acd8b1fc..38c3b3f9c3c 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"
@@ -157,6 +158,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 */
@@ -243,13 +245,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;
@@ -285,9 +291,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.
*
@@ -315,7 +341,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 82b202f3305..7b29f1814db 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 000c36d30dd..34683299620 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.
*
@@ -279,7 +286,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);
--
2.30.0.windows.2
v27-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v27-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From e391bd6c86f98e9853f3073485e50830f313a924 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 5 Feb 2025 16:40:18 +0800
Subject: [PATCH v27 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 | 52 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 441 insertions(+), 127 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 088fb175cce..3abb86cf8c2 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8065,6 +8065,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 613abcd28b7..17e87abdb94 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2387,7 +2387,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 cddc3ea9b53..7659bf8373f 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1371,7 +1371,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 2ef6594b5ce..c176bc0d87d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1190,16 +1192,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);
@@ -1209,10 +1221,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;
@@ -1238,7 +1251,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,
@@ -1276,7 +1290,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1290,12 +1304,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1326,7 +1342,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
@@ -1368,7 +1384,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1377,7 +1393,9 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
FullTransactionId next_full_xid;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
next_full_xid = ReadNextFullTransactionId();
@@ -1445,8 +1463,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 19002b52814..44ff7796b82 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4098,6 +4098,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
@@ -4507,6 +4511,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 ||
@@ -4516,7 +4529,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 02e1fdf8f78..52b94b6c018 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4868,6 +4868,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4940,11 +4941,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4977,6 +4984,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");
@@ -5010,6 +5018,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))
@@ -5268,6 +5278,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 7139c88a69a..117d5a81f1e 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index aa4363b200a..2cde9e76ca1 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6688,7 +6688,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)
{
@@ -6756,6 +6756,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 81cbf10aa28..e0be3c6fd63 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2279,8 +2279,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,8 +3690,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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v27-0004-add-a-max_conflict_retention_duration-subscripti.patchapplication/octet-stream; name=v27-0004-add-a-max_conflict_retention_duration-subscripti.patchDownload
From 7cf1ebb2603f0cc47ea7e29a99c9f26c17dfe0a2 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 15 Jan 2025 16:46:15 +0800
Subject: [PATCH v27 4/7] add a max_conflict_retention_duration subscription
option
This commit introduces the max_conflict_retention_duration subscription option,
designed to prevent excessive accumulation of dead tuples when
retain_conflict_info is enabled 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/catalogs.sgml | 10 ++
doc/src/sgml/monitoring.sgml | 13 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 40 +++++
doc/src/sgml/system-views.sgml | 11 ++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 32 +++-
src/backend/replication/logical/launcher.c | 79 ++++++++-
src/backend/replication/logical/worker.c | 144 +++++++++++++--
src/backend/replication/slot.c | 13 +-
src/bin/pg_dump/pg_dump.c | 19 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 16 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 10 ++
src/include/replication/slot.h | 2 +
src/include/replication/worker_internal.h | 6 +
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 193 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
22 files changed, 505 insertions(+), 132 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 3abb86cf8c2..d609285dcae 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8077,6 +8077,16 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflictretentionduration</structfield> <type>interger</type>
+ </para>
+ <para>
+ If not zero, the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index edc2470bcf9..0375c15f76c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2101,6 +2101,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e5415c3150d..b772cefe075 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 61374b74d1c..fef6c0757e3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -466,6 +466,46 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Specifies the maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index be81c2b51d2..c795eb21921 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -2621,6 +2621,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
perform logical decoding. It is set only for logical slots.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 39cfae43d6f..d3d9cc1779c 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retainconflictinfo = subform->subretainconflictinfo;
+ sub->maxconflictretentionduration = subform->submaxconflictretentionduration;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 7659bf8373f..930013c5984 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -980,7 +980,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);
@@ -1371,8 +1372,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretainconflictinfo, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretainconflictinfo, submaxconflictretentionduration, 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 51bd52e48a3..13199740320 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retainconflictinfo;
+ int32 maxconflictretentionduration;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_CONFLICT_INFO))
opts->retainconflictinfo = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflictretentionduration = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_CONFLICT_INFO;
opts->retainconflictinfo = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflictretentionduration = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -695,6 +709,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretainconflictinfo - 1] =
BoolGetDatum(opts.retainconflictinfo);
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1193,7 +1209,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_CONFLICT_INFO | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_CONFLICT_INFO |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1376,6 +1393,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_conflict_info = opts.retainconflictinfo;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflictretentionduration - 1] =
+ Int32GetDatum(opts.maxconflictretentionduration);
+ replaces[Anum_pg_subscription_submaxconflictretentionduration - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c176bc0d87d..0229a331d5d 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"
@@ -42,6 +43,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
@@ -458,6 +460,7 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_retention = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1170,7 +1173,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();
@@ -1199,7 +1203,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1225,22 +1229,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_retention;
SpinLockAcquire(&w->relmutex);
nonremovable_xid = w->oldest_nonremovable_xid;
+ stop_conflict_retention = w->stop_conflict_retention;
SpinLockRelease(&w->relmutex);
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers
+ * that have stopped conflict retention.
+ */
+ if (stop_conflict_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;
}
@@ -1285,12 +1299,35 @@ 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, and update the sleep time before the next
* attempt.
*/
- if (retain_conflict_info)
+ else if (nretain_conflict_info)
{
bool updated = false;
@@ -1518,7 +1555,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;
@@ -1595,6 +1632,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_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 44ff7796b82..22f2b794f62 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"
@@ -453,6 +454,7 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -3853,7 +3855,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_retention &&
+ data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval)
wait_time = Min(wait_time, data.xid_advance_interval);
rc = WaitLatchOrSocket(MyLatch,
@@ -4110,6 +4113,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_retention)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4272,6 +4279,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ bool stop_conflict_retention = false;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4333,22 +4342,72 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
- /*
- * 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);
+ /*
+ * 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();
+
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (!MySubscription->maxconflictretentionduration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ return;
- elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
- LSN_FORMAT_ARGS(data->remote_lsn),
- XidFromFullTransactionId(data->candidate_xid));
+ stop_conflict_retention = true;
+ }
+
+ if (!stop_conflict_retention)
+ {
+ /*
+ * 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));
+ }
+ else
+ {
+ /*
+ * Reaching here means the time spent applying changes up to the
+ * remote_lsn has exceeded the maximum allowed limit
+ * (max_conflict_retention_duration). So, we will stop retaining
+ * conflict information.
+ *
+ * Currently, the retention will not resume automatically unless user
+ * manually disable retain_conflict_info and re-enable it after
+ * confirming that the replication slot has been dropped.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ MyLogicalRepWorker->stop_conflict_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ MySubscription->maxconflictretentionduration));
+ }
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4412,6 +4471,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_retention = cause != RS_INVAL_NONE;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4583,6 +4687,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();
}
/*
@@ -5221,6 +5335,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 38c3b3f9c3c..9f6d520ea7a 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -108,10 +108,11 @@ const char *const SlotInvalidationCauses[] = {
[RS_INVAL_WAL_REMOVED] = "wal_removed",
[RS_INVAL_HORIZON] = "rows_removed",
[RS_INVAL_WAL_LEVEL] = "wal_level_insufficient",
+ [RS_INVAL_CONFLICT_RETENTION_DURATION] = "conflict_retention_exceeds_max_duration",
};
/* Maximum number of invalidation causes */
-#define RS_INVAL_MAX_CAUSES RS_INVAL_WAL_LEVEL
+#define RS_INVAL_MAX_CAUSES RS_INVAL_CONFLICT_RETENTION_DURATION
StaticAssertDecl(lengthof(SlotInvalidationCauses) == (RS_INVAL_MAX_CAUSES + 1),
"array length mismatch");
@@ -1572,6 +1573,11 @@ ReportSlotInvalidation(ReplicationSlotInvalidationCause cause,
case RS_INVAL_WAL_LEVEL:
appendStringInfoString(&err_detail, _("Logical decoding on standby requires \"wal_level\" >= \"logical\" on the primary server."));
break;
+
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ appendStringInfo(&err_detail, _("The duration for retaining conflict information exceeds the maximum limit defined by the subscription's max_conflict_retention_duration setting"));
+ break;
+
case RS_INVAL_NONE:
pg_unreachable();
}
@@ -1685,6 +1691,10 @@ InvalidatePossiblyObsoleteSlot(ReplicationSlotInvalidationCause cause,
if (SlotIsLogical(s))
invalidation_cause = cause;
break;
+ case RS_INVAL_CONFLICT_RETENTION_DURATION:
+ if (IsReservedSlotName(NameStr(s->data.name)))
+ invalidation_cause = cause;
+ break;
case RS_INVAL_NONE:
pg_unreachable();
}
@@ -1838,6 +1848,7 @@ InvalidatePossiblyObsoleteSlot(ReplicationSlotInvalidationCause cause,
* - RS_INVAL_HORIZON: requires a snapshot <= the given horizon in the given
* db; dboid may be InvalidOid for shared relations
* - RS_INVAL_WAL_LEVEL: is logical
+ * - RS_INVAL_CONFLICT_RETENTION_DURATION: is "pg_conflict_detection"
*
* NB - this runs as part of checkpoint, so avoid raising errors if possible.
*/
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 52b94b6c018..9d1cfe35c65 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4869,6 +4869,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretainconflictinfo;
+ int i_submaxconflictretentionduration;
int i,
ntups;
@@ -4948,10 +4949,18 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 180000)
appendPQExpBufferStr(query,
- " s.subretainconflictinfo\n");
+ " s.subretainconflictinfo,\n");
else
appendPQExpBuffer(query,
- " false AS subretainconflictinfo\n");
+ " false AS subretainconflictinfo,\n");
+
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflictretentionduration\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflictretentionduration\n");
+
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4985,6 +4994,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretainconflictinfo = PQfnumber(res, "subretainconflictinfo");
+ i_submaxconflictretentionduration = PQfnumber(res, "submaxconflictretentionduration");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5020,6 +5030,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretainconflictinfo =
(strcmp(PQgetvalue(res, i, i_subretainconflictinfo), "t") == 0);
+ subinfo[i].submaxconflictretentionduration =
+ atoi(PQgetvalue(res, i, i_submaxconflictretentionduration));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5281,6 +5293,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretainconflictinfo)
appendPQExpBufferStr(query, ", retain_conflict_info = true");
+ if (subinfo->submaxconflictretentionduration)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflictretentionduration);
+
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 117d5a81f1e..56d9cc8747c 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -682,6 +682,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretainconflictinfo;
+ int submaxconflictretentionduration;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 2cde9e76ca1..7bbc2bf6b92 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6688,7 +6688,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, false, false};
if (pset.sversion < 100000)
{
@@ -6757,10 +6757,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 180000)
+ {
appendPQExpBuffer(&buf,
", subretainconflictinfo AS \"%s\"\n",
gettext_noop("Retain conflict info"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflictretentionduration AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index e0be3c6fd63..d899428e5d5 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2278,10 +2278,10 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin", "password_required",
+ "retain_conflict_info", "run_as_owner", "slot_name",
+ "streaming", "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3689,10 +3689,10 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
- "password_required", "retain_conflict_info",
- "run_as_owner", "slot_name", "streaming",
- "synchronous_commit", "two_phase");
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin", "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_proc.dat b/src/include/catalog/pg_proc.dat
index 5b8c2ad2a54..d76799866c1 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5627,9 +5627,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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 0ac7c0b120c..f5677b53fad 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,12 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretainconflictinfo; /* True if information useful for
* conflict detection is retained */
+ int32 submaxconflictretentionduration; /* The maximum duration
+ * (in milliseconds) for
+ * which conflict
+ * information can be
+ * retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +142,10 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retainconflictinfo; /* True if information useful for conflict
* detection is retained */
+ int32 maxconflictretentionduration; /* The maximum duration (in
+ * milliseconds) for which
+ * conflict information can be
+ * 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/include/replication/slot.h b/src/include/replication/slot.h
index 34683299620..86a3e6a279b 100644
--- a/src/include/replication/slot.h
+++ b/src/include/replication/slot.h
@@ -63,6 +63,8 @@ typedef enum ReplicationSlotInvalidationCause
RS_INVAL_HORIZON,
/* wal_level insufficient for slot */
RS_INVAL_WAL_LEVEL,
+ /* duration of conflict info retention exceeds the maximum limit */
+ RS_INVAL_CONFLICT_RETENTION_DURATION,
} ReplicationSlotInvalidationCause;
extern PGDLLIMPORT const char *const SlotInvalidationCauses[];
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b09486017f4..40469f2df28 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_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 3361f6a69c9..ed2bb26b3eb 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2142,9 +2142,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,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index bff4cc051db..577493fe17b 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+------------------------------+----------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -434,19 +434,44 @@ WARNING: information for detecting conflicts cannot be fully retained when "tra
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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | t | 0 | 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
+ 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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/0
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\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 | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+----------------------+---------------------------------+--------------------+-----------------------------+----------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | 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 c65397e5ac6..17dd6aa26a9 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -303,6 +303,22 @@ ALTER SUBSCRIPTION regress_testsub SET (retain_conflict_info = false);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.30.0.windows.2
v27-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v27-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 3125f97df0be745210319cd2c970a7ef850a9b4a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 5 Feb 2025 17:15:29 +0800
Subject: [PATCH v27 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/ref/create_subscription.sgml | 5 +-
src/backend/replication/logical/launcher.c | 37 +++---
src/backend/replication/logical/worker.c | 143 ++++++++-------------
3 files changed, 77 insertions(+), 108 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fef6c0757e3..eaf2c68fc66 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -488,7 +488,10 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 0229a331d5d..dc071509e08 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -460,7 +460,8 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
- worker->stop_conflict_retention = false;
+ worker->stop_conflict_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1299,27 +1300,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1331,6 +1325,17 @@ ApplyLauncherMain(Datum main_arg)
{
bool updated = false;
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
updated = advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22f2b794f62..d8465767b3d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -454,7 +454,6 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -4113,10 +4112,6 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
if (!am_leader_apply_worker())
return;
- /* Exit early if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_retention)
- return;
-
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4279,7 +4274,9 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ TimestampTz now;
bool stop_conflict_retention = false;
+ bool status_changed;
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4342,31 +4339,36 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Check if changes up to the remote_lsn have been applied and flushed */
- if (last_flushpos < data->remote_lsn)
- {
- 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();
- /*
- * 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();
+ /*
+ * If the wait time has exceeded the maximum limit
+ * (max_conflict_retention_duration), stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (MySubscription->maxconflictretentionduration &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ MySubscription->maxconflictretentionduration))
+ stop_conflict_retention = true;
- /*
- * If the wait time has not exceeded the maximum limit
- * (max_conflict_retention_duration), continue waiting for the changes
- * to be applied. Otherwise, stop tracking the non-removable
- * transaction ID by this apply worker.
- */
- if (!MySubscription->maxconflictretentionduration ||
- !TimestampDifferenceExceeds(data->candidate_xid_time, now,
- MySubscription->maxconflictretentionduration))
- return;
+ /*
+ * Return if the apply worker is retaining conflict information, and the
+ * changes up to the remote_lsn are not yet applied and flushed.
+ */
+ if (!stop_conflict_retention && last_flushpos < data->remote_lsn)
+ return;
- stop_conflict_retention = true;
- }
+ /*
+ * Determine whether the apply worker is about to stop or resume conflict
+ * retention.
+ */
+ if (MyLogicalRepWorker->stop_conflict_retention != stop_conflict_retention)
+ status_changed = true;
if (!stop_conflict_retention)
{
@@ -4378,13 +4380,25 @@ wait_for_local_flush(RetainConflictInfoData *data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ if (status_changed)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->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),
+ MySubscription->maxconflictretentionduration));
+
elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(data->remote_lsn),
XidFromFullTransactionId(data->candidate_xid));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
}
- else
+ else if (status_changed)
{
/*
* Reaching here means the time spent applying changes up to the
@@ -4392,9 +4406,9 @@ wait_for_local_flush(RetainConflictInfoData *data)
* (max_conflict_retention_duration). So, we will stop retaining
* conflict information.
*
- * Currently, the retention will not resume automatically unless user
- * manually disable retain_conflict_info and re-enable 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.
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
@@ -4407,10 +4421,14 @@ wait_for_local_flush(RetainConflictInfoData *data)
errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
LSN_FORMAT_ARGS(data->remote_lsn),
MySubscription->maxconflictretentionduration));
- }
- /* Notify launcher to update the xmin of the conflict slot */
- ApplyLauncherWakeup();
+ /*
+ * Notify launcher to either update the xmin of the conflict slot or
+ * invalidate the slot if no other workers are retaining conflict
+ * information.
+ */
+ ApplyLauncherWakeup();
+ }
/*
* Reset all data fields except those used to determine the timing for the
@@ -4471,51 +4489,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_retention = cause != RS_INVAL_NONE;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-}
-
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4687,16 +4660,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();
}
/*
@@ -5335,8 +5298,6 @@ InitializeLogRepWorker(void)
MySubscription->name)));
CommitTransactionCommand();
-
- update_conflict_retention_status();
}
/*
--
2.30.0.windows.2
v27-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v27-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 6c9a379ac56b2c5cfc5c93b9e0805a66bf86c86f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v27 6/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 163 ++++++++++++++++++
2 files changed, 164 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..75539b2cba9
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,163 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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;");
+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;");
+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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
v27-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v27-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 850da9a21261bda870219da3508101ce6915a567 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 5 Feb 2025 11:37:49 +0800
Subject: [PATCH v27 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 17e87abdb94..2079f8081cb 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;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 0375c15f76c..722639f0711 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 930013c5984..ae47b6860dc 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1383,6 +1383,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 5f7613cc831..1e768f08c1b 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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;
@@ -431,6 +445,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 d8465767b3d..4882b1e3a9e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2745,6 +2745,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2761,15 +2764,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2780,7 +2779,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2799,19 +2798,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3132,9 +3140,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3143,20 +3151,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3165,7 +3182,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3176,7 +3193,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index e9096a88492..b6e79f554bd 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2159,7 +2159,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2181,17 +2181,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index d76799866c1..19063124ddb 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5619,9 +5619,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 45b80e6b98e..f807d6c62b5 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"
@@ -670,7 +671,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 37454dc9513..91224b6ea60 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 ed2bb26b3eb..16f32fa93f3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2151,6 +2151,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,
@@ -2158,7 +2159,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 75539b2cba9..ec446191e95 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -120,6 +125,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -127,6 +139,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
On Tue, Feb 4, 2025 at 10:30 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Feb 5, 2025 at 6:00 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jan 31, 2025 at 9:07 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I was not sure of the point of
making the max_conflict_retention_duration a per-subscription
parameter.The idea is to keep it at the same level as the other related
parameter 'retain_conflict_info'. It could be useful for cases where
publishers are from two different nodes (NP1 and NP2) and we have
separate subscriptions for both nodes. Now, it is possible that users
won't expect conflicts on the tables from one of the nodes NP1 then
she could choose to enable 'retain_conflict_info' and
'max_conflict_retention_duration' only for the subscription pointing
to publisher NP2.Now, say the publisher node that can generate conflicts (NP2) has
fewer writes and the corresponding apply worker could easily catch up
and almost always be in sync with the publisher. In contrast, the
other node that has no conflicts has a large number of writes. In such
cases, giving new options at the subscription level will be helpful.If we want to provide it at the global level, then the performance or
dead tuple control may not be any better than the current patch but
won't allow the provision for the above kinds of cases. Second, adding
two new GUCs is another thing I want to prevent. But OTOH, the
implementation could be slightly simpler if we provide these options
as GUC though I am not completely sure of that point. Having said
that, I am open to changing it to a non-subscription level. Do you
think it would be better to provide one or both of these parameters as
GUCs or do you have something else in mind?It makes sense to me to have the retain_conflict_info as a
subscription-level parameter. I was thinking of making only
max_conflict_retention_duration a global parameter, but I might be
missing something. With a subscription-level
max_conflict_retention_duration, how can users choose the setting
values for each subscription, and is there a case that can be covered
only by a subscription-level max_conflict_retention_duration?Users can configure depending on the workload of the publisher
considering the publishers are different nodes as explained in my
previous response. Also, I think it will help in resolutions where the
worker for which the duration for updating the worker_level xmin has
not exceeded the max_conflict_retention_duration can reliably detect
update_delete. Then this parameter will only be required for
subscriptions that have enabled retain_conflict_info. I am not
completely sure if these are reasons enough to keep at the
subscription level but OTOH Dilip also seems to favor keeping
max_conflict_retention_duration at susbcription-level.
I'd like to confirm what users would expect of this
max_conflict_retention_duration option and it works as expected. IIUC
users would want to use this option when they want to balance between
the reliable update_deleted conflict detection and the performance. I
think they want to detect updated_deleted reliably as much as possible
but, at the same time, would like to avoid a huge performance dip
caused by it. IOW, once the apply lag becomes larger than the limit,
they would expect to prioritize the performance (recovery) over the
reliable update_deleted conflict detection.
With the subscription-level max_conflict_retention_duration, users can
set it to '5min' to a subscription, SUB1, while not setting it to
another subscription, SUB2, (assuming here that both subscriptions set
retain_conflict_info = true). This setting works fine if SUB2 could
easily catch up while SUB1 is delaying, because in this case, SUB1
would stop updating its xmin when delaying for 5 min or longer so the
slot's xmin can advance based only on SUB2's xmin. Which is good
because it ultimately allow vacuum to remove dead tuples and
contributes to better performance. On the other hand, in cases where
SUB2 is as delayed as or more than SUB1, even if SUB1 stopped updating
its xmin, the slot's xmin would not be able to advance. IIUC
pg_conflict_detection slot won't be invalidated as long as there is at
least one subscription that sets retain_conflict_info = true and
doesn't set max_conflict_retention_duration, even if other
subscriptions set max_conflict_retention_duration.
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Fri, Feb 7, 2025 at 2:18 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I'd like to confirm what users would expect of this
max_conflict_retention_duration option and it works as expected. IIUC
users would want to use this option when they want to balance between
the reliable update_deleted conflict detection and the performance. I
think they want to detect updated_deleted reliably as much as possible
but, at the same time, would like to avoid a huge performance dip
caused by it. IOW, once the apply lag becomes larger than the limit,
they would expect to prioritize the performance (recovery) over the
reliable update_deleted conflict detection.
Yes, this understanding is correct.
With the subscription-level max_conflict_retention_duration, users can
set it to '5min' to a subscription, SUB1, while not setting it to
another subscription, SUB2, (assuming here that both subscriptions set
retain_conflict_info = true). This setting works fine if SUB2 could
easily catch up while SUB1 is delaying, because in this case, SUB1
would stop updating its xmin when delaying for 5 min or longer so the
slot's xmin can advance based only on SUB2's xmin. Which is good
because it ultimately allow vacuum to remove dead tuples and
contributes to better performance. On the other hand, in cases where
SUB2 is as delayed as or more than SUB1, even if SUB1 stopped updating
its xmin, the slot's xmin would not be able to advance. IIUC
pg_conflict_detection slot won't be invalidated as long as there is at
least one subscription that sets retain_conflict_info = true and
doesn't set max_conflict_retention_duration, even if other
subscriptions set max_conflict_retention_duration.
Right.
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.
It seems you are not convinced to provide this parameter at the
subscription level and anyway providing it as GUC will simplify the
implementation and it would probably be easier for users to configure
rather than giving it at the subscription level for all subscriptions
that have set retain_conflict_info set to true. I guess in the future
we can provide it at the subscription level as well if there is a
clear use case for it. Does that make sense to you?
--
With Regards,
Amit Kapila.
On Thu, Feb 6, 2025 at 9:47 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 7, 2025 at 2:18 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I'd like to confirm what users would expect of this
max_conflict_retention_duration option and it works as expected. IIUC
users would want to use this option when they want to balance between
the reliable update_deleted conflict detection and the performance. I
think they want to detect updated_deleted reliably as much as possible
but, at the same time, would like to avoid a huge performance dip
caused by it. IOW, once the apply lag becomes larger than the limit,
they would expect to prioritize the performance (recovery) over the
reliable update_deleted conflict detection.Yes, this understanding is correct.
With the subscription-level max_conflict_retention_duration, users can
set it to '5min' to a subscription, SUB1, while not setting it to
another subscription, SUB2, (assuming here that both subscriptions set
retain_conflict_info = true). This setting works fine if SUB2 could
easily catch up while SUB1 is delaying, because in this case, SUB1
would stop updating its xmin when delaying for 5 min or longer so the
slot's xmin can advance based only on SUB2's xmin. Which is good
because it ultimately allow vacuum to remove dead tuples and
contributes to better performance. On the other hand, in cases where
SUB2 is as delayed as or more than SUB1, even if SUB1 stopped updating
its xmin, the slot's xmin would not be able to advance. IIUC
pg_conflict_detection slot won't be invalidated as long as there is at
least one subscription that sets retain_conflict_info = true and
doesn't set max_conflict_retention_duration, even if other
subscriptions set max_conflict_retention_duration.Right.
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.It seems you are not convinced to provide this parameter at the
subscription level and anyway providing it as GUC will simplify the
implementation and it would probably be easier for users to configure
rather than giving it at the subscription level for all subscriptions
that have set retain_conflict_info set to true. I guess in the future
we can provide it at the subscription level as well if there is a
clear use case for it. Does that make sense to you?
Yes, that makes sense to me.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Fri, Feb 7, 2025 at 11:17 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 7, 2025 at 2:18 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I'd like to confirm what users would expect of this
max_conflict_retention_duration option and it works as expected. IIUC
users would want to use this option when they want to balance between
the reliable update_deleted conflict detection and the performance. I
think they want to detect updated_deleted reliably as much as possible
but, at the same time, would like to avoid a huge performance dip
caused by it. IOW, once the apply lag becomes larger than the limit,
they would expect to prioritize the performance (recovery) over the
reliable update_deleted conflict detection.Yes, this understanding is correct.
With the subscription-level max_conflict_retention_duration, users can
set it to '5min' to a subscription, SUB1, while not setting it to
another subscription, SUB2, (assuming here that both subscriptions set
retain_conflict_info = true). This setting works fine if SUB2 could
easily catch up while SUB1 is delaying, because in this case, SUB1
would stop updating its xmin when delaying for 5 min or longer so the
slot's xmin can advance based only on SUB2's xmin. Which is good
because it ultimately allow vacuum to remove dead tuples and
contributes to better performance. On the other hand, in cases where
SUB2 is as delayed as or more than SUB1, even if SUB1 stopped updating
its xmin, the slot's xmin would not be able to advance. IIUC
pg_conflict_detection slot won't be invalidated as long as there is at
least one subscription that sets retain_conflict_info = true and
doesn't set max_conflict_retention_duration, even if other
subscriptions set max_conflict_retention_duration.
That seems like a valid point.
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.It seems you are not convinced to provide this parameter at the
subscription level and anyway providing it as GUC will simplify the
implementation and it would probably be easier for users to configure
rather than giving it at the subscription level for all subscriptions
that have set retain_conflict_info set to true. I guess in the future
we can provide it at the subscription level as well if there is a
clear use case for it. Does that make sense to you?
Would it make sense to introduce a GUC parameter for this value, where
subscribers can overwrite it for their specific subscriptions, but
only up to the limit set by the GUC? This would allow flexibility in
certain cases --subscribers could opt to wait for a shorter duration
than the GUC value if needed.
Although a concrete use case isn't immediately clear, consider a
hypothetical scenario: Suppose a subscriber connected to Node1 must
wait for long period to avoid an incorrect conflict decision. In such
cases, it would rely on the default high value set by the GUC.
However, since Node1 is generally responsive and rarely has
long-running transactions, this long wait would only be necessary in
rare cases. On the other hand, a subscriber pulling from Node2 may not
require as stringent conflict detection. If Node2 frequently has
long-running transactions, waiting too long could lead to excessive
delays.
The idea here is that the Node1 subscriber can wait for the full
max_conflict_retention_duration set by the GUC when necessary, while
the Node2 subscriber can choose a shorter wait time to avoid
unnecessary delays caused by frequent long transactions.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Mon, Feb 10, 2025 at 10:26 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Feb 7, 2025 at 11:17 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.It seems you are not convinced to provide this parameter at the
subscription level and anyway providing it as GUC will simplify the
implementation and it would probably be easier for users to configure
rather than giving it at the subscription level for all subscriptions
that have set retain_conflict_info set to true. I guess in the future
we can provide it at the subscription level as well if there is a
clear use case for it. Does that make sense to you?Would it make sense to introduce a GUC parameter for this value, where
subscribers can overwrite it for their specific subscriptions, but
only up to the limit set by the GUC? This would allow flexibility in
certain cases --subscribers could opt to wait for a shorter duration
than the GUC value if needed.Although a concrete use case isn't immediately clear, consider a
hypothetical scenario: Suppose a subscriber connected to Node1 must
wait for long period to avoid an incorrect conflict decision. In such
cases, it would rely on the default high value set by the GUC.
However, since Node1 is generally responsive and rarely has
long-running transactions, this long wait would only be necessary in
rare cases. On the other hand, a subscriber pulling from Node2 may not
require as stringent conflict detection. If Node2 frequently has
long-running transactions, waiting too long could lead to excessive
delays.The idea here is that the Node1 subscriber can wait for the full
max_conflict_retention_duration set by the GUC when necessary, while
the Node2 subscriber can choose a shorter wait time to avoid
unnecessary delays caused by frequent long transactions.
I see that users can have some cases like this where it can be helpful
to provide the option to set max_conflict_retention_duration both at
GUC as well as a subscription parameter. However, I suggest let's go a
bit slower in adding more options for this particular stuff. In the
first version of this work, let's add a GUC and then let it bake for
some time after which we can discuss again adding a subscription
option based on some feedback from the field.
--
With Regards,
Amit Kapila.
On Mon, Feb 10, 2025 at 2:45 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Feb 10, 2025 at 10:26 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Feb 7, 2025 at 11:17 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.It seems you are not convinced to provide this parameter at the
subscription level and anyway providing it as GUC will simplify the
implementation and it would probably be easier for users to configure
rather than giving it at the subscription level for all subscriptions
that have set retain_conflict_info set to true. I guess in the future
we can provide it at the subscription level as well if there is a
clear use case for it. Does that make sense to you?Would it make sense to introduce a GUC parameter for this value, where
subscribers can overwrite it for their specific subscriptions, but
only up to the limit set by the GUC? This would allow flexibility in
certain cases --subscribers could opt to wait for a shorter duration
than the GUC value if needed.Although a concrete use case isn't immediately clear, consider a
hypothetical scenario: Suppose a subscriber connected to Node1 must
wait for long period to avoid an incorrect conflict decision. In such
cases, it would rely on the default high value set by the GUC.
However, since Node1 is generally responsive and rarely has
long-running transactions, this long wait would only be necessary in
rare cases. On the other hand, a subscriber pulling from Node2 may not
require as stringent conflict detection. If Node2 frequently has
long-running transactions, waiting too long could lead to excessive
delays.The idea here is that the Node1 subscriber can wait for the full
max_conflict_retention_duration set by the GUC when necessary, while
the Node2 subscriber can choose a shorter wait time to avoid
unnecessary delays caused by frequent long transactions.I see that users can have some cases like this where it can be helpful
to provide the option to set max_conflict_retention_duration both at
GUC as well as a subscription parameter. However, I suggest let's go a
bit slower in adding more options for this particular stuff. In the
first version of this work, let's add a GUC and then let it bake for
some time after which we can discuss again adding a subscription
option based on some feedback from the field.
I am fine with that.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Friday, February 7, 2025 1:47 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Feb 7, 2025 at 2:18 AM Masahiko Sawada <sawada.mshk@gmail.com>
wrote:I'd like to confirm what users would expect of this
max_conflict_retention_duration option and it works as expected. IIUC
users would want to use this option when they want to balance between
the reliable update_deleted conflict detection and the performance. I
think they want to detect updated_deleted reliably as much as possible
but, at the same time, would like to avoid a huge performance dip
caused by it. IOW, once the apply lag becomes larger than the limit,
they would expect to prioritize the performance (recovery) over the
reliable update_deleted conflict detection.Yes, this understanding is correct.
With the subscription-level max_conflict_retention_duration, users can
set it to '5min' to a subscription, SUB1, while not setting it to
another subscription, SUB2, (assuming here that both subscriptions set
retain_conflict_info = true). This setting works fine if SUB2 could
easily catch up while SUB1 is delaying, because in this case, SUB1
would stop updating its xmin when delaying for 5 min or longer so the
slot's xmin can advance based only on SUB2's xmin. Which is good
because it ultimately allow vacuum to remove dead tuples and
contributes to better performance. On the other hand, in cases where
SUB2 is as delayed as or more than SUB1, even if SUB1 stopped updating
its xmin, the slot's xmin would not be able to advance. IIUC
pg_conflict_detection slot won't be invalidated as long as there is at
least one subscription that sets retain_conflict_info = true and
doesn't set max_conflict_retention_duration, even if other
subscriptions set max_conflict_retention_duration.Right.
I'm not really sure that these behaviors are the expected behavior of
users who set max_conflict_retention_duration to some subscriptions.
Or I might have set the wrong expectation or assumption on this
parameter. I'm fine with a subscription-level
max_conflict_retention_duration if it's clear this option works as
expected by users who want to use it.
Here is the v28 patch set, which converts the subscription option
max_conflict_retention_duration into a GUC. Other logic remains unchanged.
Best Regards,
Hou zj
Attachments:
v28-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v28-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 834702b565c2476b72d1366004f188b943874e67 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 5 Feb 2025 16:40:18 +0800
Subject: [PATCH v28 3/4] 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 | 52 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 27 +++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 441 insertions(+), 127 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index ee59a7e15d0..00fc5dac170 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8067,6 +8067,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 3d18e507bbc..632c102c2ae 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2387,7 +2387,8 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e95e0cd2715..61374b74d1c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 eff0990957e..8afbba16f8b 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1373,7 +1373,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 3cf1e055394..51bd52e48a3 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();
{
@@ -2193,6 +2267,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 2ef6594b5ce..c176bc0d87d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -163,6 +163,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);
@@ -1169,6 +1170,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();
@@ -1190,16 +1192,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);
@@ -1209,10 +1221,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;
@@ -1238,7 +1251,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,
@@ -1276,7 +1290,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1290,12 +1304,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1326,7 +1342,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
@@ -1368,7 +1384,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1377,7 +1393,9 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
FullTransactionId next_full_xid;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
next_full_xid = ReadNextFullTransactionId();
@@ -1445,8 +1463,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 74e39c231ed..923df059ff5 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4111,6 +4111,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
@@ -4520,6 +4524,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 ||
@@ -4529,7 +4542,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 6370bb711c0..d2a2f738e70 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4857,6 +4857,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -4929,11 +4930,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -4966,6 +4973,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");
@@ -4999,6 +5007,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))
@@ -5257,6 +5267,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 7139c88a69a..117d5a81f1e 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -681,6 +681,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 7ca1d8fffc9..70489b93745 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1845,7 +1845,13 @@ check_new_cluster_subscription_configuration(void)
pg_fatal("could not determine parameter settings on new cluster");
max_replication_slots = atoi(PQgetvalue(res, 0, 0));
- if (old_cluster.nsubs > max_replication_slots)
+
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+ else if (old_cluster.nsubs > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"subscriptions (%d) on the old cluster",
max_replication_slots, old_cluster.nsubs);
@@ -1912,6 +1918,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 ad52de8b607..1f84ed292a5 100644
--- a/src/bin/pg_upgrade/info.c
+++ b/src/bin/pg_upgrade/info.c
@@ -750,20 +750,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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 0cdd675e4f1..58395ab31cf 100644
--- a/src/bin/pg_upgrade/pg_upgrade.h
+++ b/src/bin/pg_upgrade/pg_upgrade.h
@@ -295,6 +295,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;
@@ -430,7 +432,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/psql/describe.c b/src/bin/psql/describe.c
index 3b7ba66fad0..8c7ef1ba80b 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6747,7 +6747,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)
{
@@ -6815,6 +6815,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 eb8bc128720..2097267cdef 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2297,8 +2297,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3718,8 +3719,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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v28-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v28-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From cbd5b287964e9527963f9cc9547dfbe7ed99fa2e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 20 Feb 2025 14:53:43 +0800
Subject: [PATCH v28 4/4] 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 | 144 ++++++++++++++++--
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, 312 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 1041819b500..6decdc89eef 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5215,6 +5215,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 71c4f96d054..f5c1bba496a 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2101,6 +2101,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 3f5a306247e..8e8bef7c801 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -2626,6 +2626,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 8afbba16f8b..a0695ac7247 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -980,7 +980,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 c176bc0d87d..af4cd864d97 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"
@@ -42,6 +43,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
@@ -62,6 +64,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;
@@ -458,6 +461,7 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_retention = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1170,7 +1174,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();
@@ -1199,7 +1204,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1225,22 +1230,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_retention;
SpinLockAcquire(&w->relmutex);
nonremovable_xid = w->oldest_nonremovable_xid;
+ stop_conflict_retention = w->stop_conflict_retention;
SpinLockRelease(&w->relmutex);
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers
+ * that have stopped conflict retention.
+ */
+ if (stop_conflict_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;
}
@@ -1285,12 +1300,35 @@ 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, and update the sleep time before the next
* attempt.
*/
- if (retain_conflict_info)
+ else if (nretain_conflict_info)
{
bool updated = false;
@@ -1518,7 +1556,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;
@@ -1595,6 +1633,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_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 923df059ff5..6b1746e0e2b 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"
@@ -453,6 +454,7 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -3866,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_retention &&
+ data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval)
wait_time = Min(wait_time, data.xid_advance_interval);
rc = WaitLatchOrSocket(MyLatch,
@@ -4123,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_retention)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4285,6 +4292,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ bool stop_conflict_retention = false;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4346,22 +4355,72 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
- /*
- * 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);
+ /*
+ * 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();
+
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (!max_conflict_retention_duration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ max_conflict_retention_duration))
+ return;
- elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
- LSN_FORMAT_ARGS(data->remote_lsn),
- XidFromFullTransactionId(data->candidate_xid));
+ stop_conflict_retention = true;
+ }
+
+ if (!stop_conflict_retention)
+ {
+ /*
+ * 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));
+ }
+ else
+ {
+ /*
+ * Reaching here means the time spent applying changes up to the
+ * remote_lsn has exceeded the maximum allowed limit
+ * (max_conflict_retention_duration). So, we will stop retaining
+ * conflict information.
+ *
+ * Currently, the retention will not resume automatically unless user
+ * manually disable retain_conflict_info and re-enable it after
+ * confirming that the replication slot has been dropped.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ MyLogicalRepWorker->stop_conflict_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ max_conflict_retention_duration));
+ }
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4425,6 +4484,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_retention = cause != RS_INVAL_NONE;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4596,6 +4700,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();
}
/*
@@ -5234,6 +5348,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 e0a53031895..0eedc3016bf 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 3cde94a1759..745a46a989e 100644
--- a/src/backend/utils/misc/guc_tables.c
+++ b/src/backend/utils/misc/guc_tables.c
@@ -3311,6 +3311,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 415f253096c..5b008e43556 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -381,6 +381,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 9e803d610d7..f388205d741 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5633,9 +5633,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 7b29f1814db..d347d3d67f6 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 e4f1e69cb6b..155ee176e0b 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 b09486017f4..40469f2df28 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_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 5baba8d39ff..892ebb1136e 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,9 +2144,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,
--
2.30.0.windows.2
v28-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v28-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From ea45e57d44cbe15d9166e157e7601e83184ee477 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 13 Feb 2025 14:08:11 +0800
Subject: [PATCH v28 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 | 140 ++++++++-------------
3 files changed, 77 insertions(+), 105 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 6decdc89eef..04067b0d437 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5239,7 +5239,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index af4cd864d97..4d92efd9e77 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -461,7 +461,8 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
- worker->stop_conflict_retention = false;
+ worker->stop_conflict_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1300,27 +1301,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1332,6 +1326,17 @@ ApplyLauncherMain(Datum main_arg)
{
bool updated = false;
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
updated = advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6b1746e0e2b..548ebb37593 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -454,7 +454,6 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -4126,10 +4125,6 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
if (!am_leader_apply_worker())
return;
- /* Exit early if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_retention)
- return;
-
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4292,7 +4287,9 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ TimestampTz now;
bool stop_conflict_retention = false;
+ bool status_changed;
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4355,31 +4352,36 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Check if changes up to the remote_lsn have been applied and flushed */
- if (last_flushpos < data->remote_lsn)
- {
- 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();
- /*
- * 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();
+ /*
+ * If the wait time has exceeded the maximum limit
+ * (max_conflict_retention_duration), stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (max_conflict_retention_duration &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ max_conflict_retention_duration))
+ stop_conflict_retention = true;
- /*
- * If the wait time has not exceeded the maximum limit
- * (max_conflict_retention_duration), continue waiting for the changes
- * to be applied. Otherwise, stop tracking the non-removable
- * transaction ID by this apply worker.
- */
- if (!max_conflict_retention_duration ||
- !TimestampDifferenceExceeds(data->candidate_xid_time, now,
- max_conflict_retention_duration))
- return;
+ /*
+ * Return if the apply worker is retaining conflict information, and the
+ * changes up to the remote_lsn are not yet applied and flushed.
+ */
+ if (!stop_conflict_retention && last_flushpos < data->remote_lsn)
+ return;
- stop_conflict_retention = true;
- }
+ /*
+ * Determine whether the apply worker is about to stop or resume conflict
+ * retention.
+ */
+ if (MyLogicalRepWorker->stop_conflict_retention != stop_conflict_retention)
+ status_changed = true;
if (!stop_conflict_retention)
{
@@ -4391,13 +4393,25 @@ wait_for_local_flush(RetainConflictInfoData *data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ if (status_changed)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->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));
+
elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(data->remote_lsn),
XidFromFullTransactionId(data->candidate_xid));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
}
- else
+ else if (status_changed)
{
/*
* Reaching here means the time spent applying changes up to the
@@ -4405,9 +4419,9 @@ wait_for_local_flush(RetainConflictInfoData *data)
* (max_conflict_retention_duration). So, we will stop retaining
* conflict information.
*
- * Currently, the retention will not resume automatically unless user
- * manually disable retain_conflict_info and re-enable 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.
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
@@ -4420,6 +4434,13 @@ wait_for_local_flush(RetainConflictInfoData *data)
errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
LSN_FORMAT_ARGS(data->remote_lsn),
max_conflict_retention_duration));
+
+ /*
+ * Notify launcher to either update the xmin of the conflict slot or
+ * invalidate the slot if no other workers are retaining conflict
+ * information.
+ */
+ ApplyLauncherWakeup();
}
/* Notify launcher to update the xmin of the conflict slot */
@@ -4484,51 +4505,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_retention = cause != RS_INVAL_NONE;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-}
-
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4700,16 +4676,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();
}
/*
@@ -5348,8 +5314,6 @@ InitializeLogRepWorker(void)
MySubscription->name)));
CommitTransactionCommand();
-
- update_conflict_retention_status();
}
/*
--
2.30.0.windows.2
v28-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v28-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 4878b0e8b6294be817a6662af4ddb95eee80d046 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 6 Jan 2025 15:56:23 +0800
Subject: [PATCH v28 6/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/meson.build | 1 +
.../t/035_retain_conflict_info.pl | 163 ++++++++++++++++++
2 files changed, 164 insertions(+)
create mode 100644 src/test/subscription/t/035_retain_conflict_info.pl
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index d40b49714f6..8bf4a83ea67 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -41,6 +41,7 @@ tests += {
't/032_subscribe_use_index.pl',
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
+ 't/035_retain_conflict_info.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
new file mode 100644
index 00000000000..75539b2cba9
--- /dev/null
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -0,0 +1,163 @@
+
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test the management of the replication slot 'pg_conflict_detection'.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+my $subname_AB = 'tap_sub_a_b';
+my $subname_BA = 'tap_sub_b_a';
+
+###############################################################################
+# 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 = PostgreSQL::Test::Cluster->new('node_A');
+$node_A->init(allows_streaming => 'logical');
+$node_A->append_conf(
+ 'postgresql.conf',
+ qq{autovacuum = off
+ log_min_messages = 'debug2'});
+$node_A->start;
+
+# node_B
+my $node_B = PostgreSQL::Test::Cluster->new('node_B');
+$node_B->init(allows_streaming => 'logical');
+$node_B->append_conf('postgresql.conf', "track_commit_timestamp = on");
+$node_B->start;
+
+# 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)");
+
+# 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;");
+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;");
+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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
v28-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v28-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 11b2f444ea56f82c59a2e29e10b037c7dd3ee96d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 5 Feb 2025 11:37:49 +0800
Subject: [PATCH v28 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 | 61 +++++---
src/backend/utils/adt/pgstatfuncs.c | 16 +-
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 +-
.../t/035_retain_conflict_info.pl | 23 ++-
12 files changed, 265 insertions(+), 41 deletions(-)
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 632c102c2ae..9d40d8e1677 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;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index f5c1bba496a..7f6ee9719e0 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index a0695ac7247..9f2708dd9a7 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1385,6 +1385,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 5f7613cc831..1e768f08c1b 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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;
@@ -431,6 +445,122 @@ retry:
return found;
}
+/*
+ * 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 772fc83e88b..0eedbd2b163 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",
@@ -170,6 +171,7 @@ errcode_apply_conflict(ConflictType type)
case CT_INSERT_EXISTS:
case CT_UPDATE_EXISTS:
return errcode(ERRCODE_UNIQUE_VIOLATION);
+ case CT_UPDATE_DELETED:
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
@@ -240,6 +242,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 548ebb37593..c1abd1f6fbd 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2753,6 +2753,9 @@ apply_handle_update_internal(ApplyExecutionData *edata,
TupleTableSlot *localslot;
bool found;
MemoryContext oldctx;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
EvalPlanQualInit(&epqstate, estate, NULL, NIL, -1, NIL);
ExecOpenIndices(relinfo, true);
@@ -2769,15 +2772,11 @@ apply_handle_update_internal(ApplyExecutionData *edata,
*/
if (found)
{
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
-
/*
* Report the conflict if the tuple was modified by a different
* origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -2788,7 +2787,7 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ReportApplyConflict(estate, relinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot, localslot, newslot,
- InvalidOid, localxmin, localorigin, localts);
+ InvalidOid, localxid, localorigin, localts);
}
/* Process and store remote tuple in the slot */
@@ -2807,19 +2806,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, relinfo, LOG, type,
remoteslot, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid, localorigin, localts);
}
/* Cleanup. */
@@ -3149,9 +3157,9 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
Relation partrel_new;
bool found;
EPQState epqstate;
- RepOriginId localorigin;
- TransactionId localxmin;
- TimestampTz localts;
+ RepOriginId localorigin = InvalidRepOriginId;
+ TransactionId localxid = InvalidTransactionId;
+ TimestampTz localts = 0;
/* Get the matching local tuple from the partition. */
found = FindReplTupleInLocalRel(edata, partrel,
@@ -3160,20 +3168,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retainconflictinfo &&
+ FindMostRecentlyDeletedTupleInfo(partrel, remoteslot_part,
+ &localxid, &localorigin,
+ &localts) &&
+ localorigin != 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,
+ ReportApplyConflict(estate, partrelinfo, LOG, type,
remoteslot_part, NULL, newslot,
- InvalidOid, InvalidTransactionId,
- InvalidRepOriginId, 0);
+ InvalidOid, localxid,
+ localorigin, localts);
return;
}
@@ -3182,7 +3199,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
* Report the conflict if the tuple was modified by a
* different origin.
*/
- if (GetTupleTransactionInfo(localslot, &localxmin, &localorigin, &localts) &&
+ if (GetTupleTransactionInfo(localslot, &localxid, &localorigin, &localts) &&
localorigin != replorigin_session_origin)
{
TupleTableSlot *newslot;
@@ -3193,7 +3210,7 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ReportApplyConflict(estate, partrelinfo, LOG, CT_UPDATE_ORIGIN_DIFFERS,
remoteslot_part, localslot, newslot,
- InvalidOid, localxmin, localorigin,
+ InvalidOid, localxid, localorigin,
localts);
}
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index e9096a88492..b6e79f554bd 100644
--- a/src/backend/utils/adt/pgstatfuncs.c
+++ b/src/backend/utils/adt/pgstatfuncs.c
@@ -2159,7 +2159,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 10
+#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11
Oid subid = PG_GETARG_OID(0);
TupleDesc tupdesc;
Datum values[PG_STAT_GET_SUBSCRIPTION_STATS_COLS] = {0};
@@ -2181,17 +2181,19 @@ 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, "stats_reset",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 10, "confl_delete_missing",
+ INT8OID, -1, 0);
+ TupleDescInitEntry(tupdesc, (AttrNumber) 11, "stats_reset",
TIMESTAMPTZOID, -1, 0);
BlessTupleDesc(tupdesc);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index f388205d741..90dace4d421 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5625,9 +5625,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,timestamptz}',
- proargmodes => '{i,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,stats_reset}',
+ 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_deleted,confl_update_origin_differs,confl_update_exists,confl_update_missing,confl_delete_origin_differs,confl_delete_missing,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 30e2a82346f..f614f35db55 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"
@@ -671,7 +672,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 37454dc9513..91224b6ea60 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 892ebb1136e..a6388eb9405 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2153,6 +2153,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,
@@ -2160,7 +2161,7 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_delete_missing,
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, 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, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
diff --git a/src/test/subscription/t/035_retain_conflict_info.pl b/src/test/subscription/t/035_retain_conflict_info.pl
index 75539b2cba9..ec446191e95 100644
--- a/src/test/subscription/t/035_retain_conflict_info.pl
+++ b/src/test/subscription/t/035_retain_conflict_info.pl
@@ -1,7 +1,9 @@
# Copyright (c) 2025, PostgreSQL Global Development Group
-# Test the management of the replication slot 'pg_conflict_detection'.
+# Test the CREATE SUBSCRIPTION 'retain_conflict_info' parameter and its
+# interaction with the management of the replication slot
+# 'pg_conflict_detection'.
use strict;
use warnings FATAL => 'all';
use PostgreSQL::Test::Cluster;
@@ -15,7 +17,9 @@ my $subname_BA = 'tap_sub_b_a';
# 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
@@ -24,7 +28,8 @@ my $node_A = PostgreSQL::Test::Cluster->new('node_A');
$node_A->init(allows_streaming => 'logical');
$node_A->append_conf(
'postgresql.conf',
- qq{autovacuum = off
+ qq{track_commit_timestamp = on
+ autovacuum = off
log_min_messages = 'debug2'});
$node_A->start;
@@ -120,6 +125,13 @@ ok( $stderr =~
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.*\n.*DETAIL:.* Deleting the row that was modified locally in transaction [0-9]+ at .*\n.*Existing local tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'delete target row was modified in tab');
+
+$log_location = -s $node_A->logfile;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -127,6 +139,11 @@ $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.*\n.*DETAIL:.* The row to be updated was deleted locally in transaction [0-9]+ at .*\n.*Remote tuple \(1, 3\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
v28-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v28-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 7e9da4f2427cca7784e1fdb504450e6987071cb1 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v28 1/4] 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 fb5dec1172e..49a4d3ab1b8 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2450,6 +2450,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2594,6 +2657,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..41fb4fc5025 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 1b4f21a88d3..93668a81a34 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,9 +1431,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.
@@ -1536,7 +1536,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 a3c7adbf1a8..31ebef831c2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 31ab69ea13a..82d5515b0cd 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,
@@ -3586,6 +3661,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
@@ -3664,6 +3740,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);
@@ -3690,6 +3768,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3705,8 +3785,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);
@@ -3719,6 +3823,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)
{
/*
@@ -3753,6 +3862,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,
@@ -3816,6 +3932,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
@@ -3851,7 +3969,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;
@@ -3929,6 +4046,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 446d10c1a7d..9d27121a2e4 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -83,6 +83,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"
@@ -253,6 +254,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);
@@ -2313,6 +2315,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2659,6 +2665,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 2e54c11f880..064841b482d 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 30b2775952c..b09486017f4 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 20777f7d5ae..d2452904ddb 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -121,9 +121,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 ef0b733ebe8..bd8b17a6d0d 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 fb39c915d76..0bc5552b2b1 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2483,6 +2483,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v28-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v28-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 97222a2c0d15734e421606840e0ea42bcd9e778e Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v28 2/4] 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 | 227 +++++++++++++++++-
.../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, 292 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 9eedcf6f0f4..1041819b500 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4782,6 +4782,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index df32ee0bf5b..44b7c56111b 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29269,7 +29269,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29314,7 +29316,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29344,6 +29348,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29365,7 +29371,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 49a4d3ab1b8..e20b48a49ac 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2032,6 +2032,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6cf7d4f9a1a..e95e0cd2715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index f234007d348..0969057b0d1 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4752,7 +4752,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 2d8a71ca1e1..3cf1e055394 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 31ebef831c2..2ef6594b5ce 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -46,6 +46,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +103,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 +111,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1121,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1137,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1168,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 +1189,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 +1271,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1325,138 @@ 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 bool
+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 false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 b42f4002ba8..0c3b955a441 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4788,7 +4788,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 82d5515b0cd..74e39c231ed 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4359,6 +4359,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 292407f5149..e0a53031895 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 82b202f3305..7b29f1814db 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 f5a24ccfbf2..e4f1e69cb6b 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);
--
2.30.0.windows.2
On Thu, 20 Feb 2025 at 12:50, Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the v28 patch set, which converts the subscription option
max_conflict_retention_duration into a GUC. Other logic remains unchanged.
After discussing with Hou internally, I have moved this to the next
CommitFest since it will not be committed in the current release. This
also allows reviewers to focus on the remaining patches in the current
CommitFest.
Regards,
Vignesh
On Wed, Mar 12, 2025 at 7:36 PM vignesh C wrote:
On Thu, 20 Feb 2025 at 12:50, Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is the v28 patch set, which converts the subscription option
max_conflict_retention_duration into a GUC. Other logic remainsunchanged.
After discussing with Hou internally, I have moved this to the next CommitFest
since it will not be committed in the current release. This also allows reviewers
to focus on the remaining patches in the current CommitFest.
Thanks!
Here's a rebased version of the patch series.
Best Regards,
Hou zj
Attachments:
v29-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v29-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 42d2bd7ab50f1e426bf380ed14808684b07cc64a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v29 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 04d8e7d21af..6baa76429d9 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2565,6 +2565,69 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2709,6 +2772,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..41fb4fc5025 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 b885513f765..dfc5108da3c 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 10677da56b2..8e42787a426 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 e3b2b144942..0c592b10e09 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,
@@ -3582,6 +3657,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
@@ -3660,6 +3736,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);
@@ -3686,6 +3764,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&data, false);
}
else if (c == 'k')
{
@@ -3701,8 +3781,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);
@@ -3715,6 +3819,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)
{
/*
@@ -3749,6 +3858,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,
@@ -3812,6 +3928,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
@@ -3847,7 +3965,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;
@@ -3925,6 +4042,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 1028919aecb..6c7d6c9b5a7 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"
@@ -254,6 +255,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);
@@ -2315,6 +2317,10 @@ ProcessStandbyMessage(void)
ProcessStandbyHSFeedbackMessage();
break;
+ case 'p':
+ ProcessStandbyPSRequestMessage();
+ break;
+
default:
ereport(COMMERROR,
(errcode(ERRCODE_PROTOCOL_VIOLATION),
@@ -2661,6 +2667,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 e5b945a9ee3..19dd8d7a8b7 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 30b2775952c..b09486017f4 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 f51b03d3822..58baf38d590 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -122,9 +122,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 ef0b733ebe8..bd8b17a6d0d 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 3fbf5a4c212..d99b1765e46 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2531,6 +2531,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v29-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v29-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 78931fdd4952aa10ef9dbdca8570ea50739ed62b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v29 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 | 228 +++++++++++++++++-
.../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, 293 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 69fc93dffc4..f5c0c6b2493 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4899,6 +4899,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index f8c1deb04ee..b8a8ab6a87c 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29425,7 +29425,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29469,7 +29471,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29499,6 +29503,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29520,7 +29526,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 6baa76429d9..c095971beee 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2147,6 +2147,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..eec85cde880 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 2c19013c98b..0ff8c3ae046 100644
--- a/src/backend/access/transam/xlogrecovery.c
+++ b/src/backend/access/transam/xlogrecovery.c
@@ -4754,7 +4754,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 4aec73bcc6b..46d4e65da97 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 8e42787a426..86c1a191dc2 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"
@@ -46,6 +47,18 @@
/* max sleep time between cycles (3min) */
#define DEFAULT_NAPTIME_PER_CYCLE 180000L
+/*
+ * Min sleep time (200ms) between cycles to update the xmin value of the
+ * replication slot.
+ */
+#define MIN_NAPTIME_PER_SLOT_UPDATE 200
+
+/*
+ * Max sleep time between xmin update cycles (30 seconds) if any
+ * subscription has retain_conflict_info set to true.
+ */
+#define MAX_NAPTIME_FOR_SLOT_UPDATE 30000L
+
/* GUC variables */
int max_logical_replication_workers = 4;
int max_sync_workers_per_subscription = 2;
@@ -91,7 +104,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 +112,10 @@ 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 bool advance_conflict_slot_xmin(FullTransactionId new_xmin);
+static void compute_slot_update_naptime(bool slot_updated, long *sleep_time);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1122,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1119,6 +1138,9 @@ ApplyLauncherWakeup(void)
void
ApplyLauncherMain(Datum main_arg)
{
+ bool slot_maybe_exist = true;
+ long slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+
ereport(DEBUG1,
(errmsg_internal("logical replication launcher started")));
@@ -1147,6 +1169,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 +1190,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 +1272,33 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed, and update the sleep time before the next
+ * attempt.
+ */
+ if (sublist)
+ {
+ bool updated = false;
+
+ if (can_advance_xmin)
+ updated = advance_conflict_slot_xmin(xmin);
+
+ compute_slot_update_naptime(updated, &slot_update_wait_time);
+ wait_time = Min(wait_time, slot_update_wait_time);
+
+ 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 +1326,138 @@ 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 bool
+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 false;
+
+ 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 true;
+}
+
+/*
+ * Update the sleep time before the next slot update.
+ *
+ * If there is no slot activity, the wait time between sync cycles will double
+ * (up to a maximum of 30 seconds). If there is some slot activity, the wait
+ * time between sync cycles is reset to the minimum (200ms).
+ */
+static void
+compute_slot_update_naptime(bool slot_updated, long *sleep_time)
+{
+
+ if (!slot_updated)
+ {
+ /*
+ * The slot was not updated, so double the sleep time, but not beyond
+ * the maximum allowable value.
+ */
+ *sleep_time = Min(*sleep_time * 2, MAX_NAPTIME_FOR_SLOT_UPDATE);
+ }
+ else
+ {
+ /*
+ * The slot was updated since the last sleep, so reset the sleep time.
+ */
+ *sleep_time = MIN_NAPTIME_PER_SLOT_UPDATE;
+ }
+}
+
+/*
+ * 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 977fbcd2474..ddf5dea602a 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 0c592b10e09..095a48f6075 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4355,6 +4355,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 719e531eb90..13b9396dffb 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 82b202f3305..7b29f1814db 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 f5a24ccfbf2..e4f1e69cb6b 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);
--
2.30.0.windows.2
v29-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v29-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 5767024e11df20ba2187d48ea6be4678cb6ba5e4 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 26 Mar 2025 15:52:01 +0800
Subject: [PATCH v29 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 | 52 ++++--
src/backend/replication/logical/worker.c | 16 +-
src/bin/pg_dump/pg_dump.c | 17 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 38 ++++-
src/bin/pg_upgrade/info.c | 25 ++-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
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 ++
19 files changed, 448 insertions(+), 131 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fb050635551..f3f885ff3a6 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8088,6 +8088,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index f288c049a5c..9fcb7c0ff73 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
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index eec85cde880..b7ec696eb3c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 31d269b7ee0..769962f5c42 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1369,7 +1369,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 46d4e65da97..47ba22d28b8 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 86c1a191dc2..fc453d8c422 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -164,6 +164,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);
@@ -1170,6 +1171,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();
@@ -1191,16 +1193,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);
@@ -1210,10 +1222,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;
@@ -1239,7 +1252,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,
@@ -1277,7 +1291,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed, and update the sleep time before the next
* attempt.
*/
- if (sublist)
+ if (retain_conflict_info)
{
bool updated = false;
@@ -1291,12 +1305,14 @@ 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)
{
drop_conflict_slot_if_exists();
slot_maybe_exist = false;
+ slot_update_wait_time = MIN_NAPTIME_PER_SLOT_UPDATE;
}
/* Switch back to original memory context. */
@@ -1327,7 +1343,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
@@ -1369,7 +1385,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 bool
advance_conflict_slot_xmin(FullTransactionId new_xmin)
@@ -1378,7 +1394,9 @@ advance_conflict_slot_xmin(FullTransactionId new_xmin)
FullTransactionId next_full_xid;
Assert(MyReplicationSlot);
- Assert(FullTransactionIdIsValid(new_xmin));
+
+ if (!FullTransactionIdIsValid(new_xmin))
+ return false;
next_full_xid = ReadNextFullTransactionId();
@@ -1446,8 +1464,8 @@ compute_slot_update_naptime(bool slot_updated, long *sleep_time)
}
/*
- * 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 095a48f6075..6e3dcf295b1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4107,6 +4107,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
@@ -4516,6 +4520,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 ||
@@ -4525,7 +4538,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 e41e645f649..04ee5a3d2fa 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4942,6 +4942,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5014,11 +5015,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 170000)
appendPQExpBufferStr(query,
- " s.subfailover\n");
+ " s.subfailover,\n");
else
appendPQExpBuffer(query,
- " false AS subfailover\n");
+ " false AS subfailover,\n");
+ if (fout->remoteVersion >= 180000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBuffer(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5051,6 +5058,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");
@@ -5084,6 +5092,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))
@@ -5342,6 +5352,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 bbdb30b5f54..f357e6c9aef 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -703,6 +703,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 02d9146e5ed..26c6fd076ee 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -627,7 +627,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();
}
@@ -1829,9 +1829,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)
@@ -1839,6 +1840,7 @@ check_new_cluster_subscription_configuration(void)
PGresult *res;
PGconn *conn;
int max_active_replication_origins;
+ int max_replication_slots;
/* Subscriptions and their dependencies can be migrated since PG17. */
if (GET_MAJOR_VERSION(old_cluster.major_version) < 1700)
@@ -1853,9 +1855,9 @@ check_new_cluster_subscription_configuration(void)
conn = connectToServer(&new_cluster, "template1");
res = executeQueryOrDie(conn, "SELECT setting FROM pg_settings "
- "WHERE name = 'max_active_replication_origins';");
+ "WHERE name IN ('max_active_replication_origins', 'max_replication_slots');");
- if (PQntuples(res) != 1)
+ if (PQntuples(res) != 2)
pg_fatal("could not determine parameter settings on new cluster");
max_active_replication_origins = atoi(PQgetvalue(res, 0, 0));
@@ -1864,6 +1866,13 @@ check_new_cluster_subscription_configuration(void)
"subscriptions (%d) on the old cluster",
max_active_replication_origins, old_cluster.nsubs);
+ max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ old_cluster.nsubs + 1 > max_replication_slots)
+ pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
+ "subscriptions plus one (%d) on the old cluster",
+ max_replication_slots, old_cluster.nsubs + 1);
+
PQclear(res);
PQfinish(conn);
@@ -1926,6 +1935,23 @@ 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 (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ 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 4b7a56f5b3b..64bad04c841 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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/psql/describe.c b/src/bin/psql/describe.c
index bf565afcc4e..86a67b7f196 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6750,7 +6750,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)
{
@@ -6818,6 +6818,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 98951aef82c..5cd75d6a671 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 <name> 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 <name> 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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v29-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v29-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 9c38ab1e5dca4e427bef833562a0d21a6128b499 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 20 Feb 2025 14:53:43 +0800
Subject: [PATCH v29 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 | 144 ++++++++++++++++--
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, 312 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index f5c0c6b2493..5232a9fa844 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5327,6 +5327,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 0960f5ba94a..6636459b870 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2107,6 +2107,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 3f5a306247e..8e8bef7c801 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -2626,6 +2626,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 769962f5c42..5c30ac999ba 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -980,7 +980,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 fc453d8c422..4ffd3ba4692 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
@@ -63,6 +65,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;
@@ -459,6 +462,7 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_retention = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1171,7 +1175,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();
@@ -1200,7 +1205,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1226,22 +1231,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_retention;
SpinLockAcquire(&w->relmutex);
nonremovable_xid = w->oldest_nonremovable_xid;
+ stop_conflict_retention = w->stop_conflict_retention;
SpinLockRelease(&w->relmutex);
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers
+ * that have stopped conflict retention.
+ */
+ if (stop_conflict_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;
}
@@ -1286,12 +1301,35 @@ 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, and update the sleep time before the next
* attempt.
*/
- if (retain_conflict_info)
+ else if (nretain_conflict_info)
{
bool updated = false;
@@ -1519,7 +1557,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;
@@ -1596,6 +1634,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_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 6e3dcf295b1..82d03b5725f 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"
@@ -453,6 +454,7 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -3862,7 +3864,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_retention &&
+ data.phase == RCI_GET_CANDIDATE_XID && data.xid_advance_interval)
wait_time = Min(wait_time, data.xid_advance_interval);
rc = WaitLatchOrSocket(MyLatch,
@@ -4119,6 +4122,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_retention)
+ return;
+
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4281,6 +4288,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ bool stop_conflict_retention = false;
+
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4342,22 +4351,72 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Return to wait for the changes to be applied */
+ /* Check if changes up to the remote_lsn have been applied and flushed */
if (last_flushpos < data->remote_lsn)
- return;
+ {
+ TimestampTz now;
- /*
- * 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);
+ /*
+ * 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();
+
+ /*
+ * If the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration), continue waiting for the changes
+ * to be applied. Otherwise, stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (!max_conflict_retention_duration ||
+ !TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ max_conflict_retention_duration))
+ return;
- elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
- LSN_FORMAT_ARGS(data->remote_lsn),
- XidFromFullTransactionId(data->candidate_xid));
+ stop_conflict_retention = true;
+ }
+
+ if (!stop_conflict_retention)
+ {
+ /*
+ * 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));
+ }
+ else
+ {
+ /*
+ * Reaching here means the time spent applying changes up to the
+ * remote_lsn has exceeded the maximum allowed limit
+ * (max_conflict_retention_duration). So, we will stop retaining
+ * conflict information.
+ *
+ * Currently, the retention will not resume automatically unless user
+ * manually disable retain_conflict_info and re-enable it after
+ * confirming that the replication slot has been dropped.
+ */
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ MyLogicalRepWorker->stop_conflict_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ max_conflict_retention_duration));
+ }
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4421,6 +4480,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_retention = cause != RS_INVAL_NONE;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4592,6 +4696,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();
}
/*
@@ -5230,6 +5344,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 13b9396dffb..392a35edf07 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 989825d3a9c..8d926ee4cdc 100644
--- a/src/backend/utils/misc/guc_tables.c
+++ b/src/backend/utils/misc/guc_tables.c
@@ -3386,6 +3386,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 0b9e3066bde..309e0cacf24 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -395,6 +395,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 3f7b82e02bb..8a61ed40f6d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5655,9 +5655,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 7b29f1814db..d347d3d67f6 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 e4f1e69cb6b..155ee176e0b 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 b09486017f4..40469f2df28 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_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 47478969135..e6a2eff3961 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2144,9 +2144,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,
--
2.30.0.windows.2
v29-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v29-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 0a2d120e003511f419f25be118fb087f3321c9d5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 13 Feb 2025 14:08:11 +0800
Subject: [PATCH v29 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 | 140 ++++++++-------------
3 files changed, 77 insertions(+), 105 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 5232a9fa844..1657e9fd8d2 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5351,7 +5351,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4ffd3ba4692..aa49aa0bd14 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -462,7 +462,8 @@ retry:
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
worker->oldest_nonremovable_xid = InvalidFullTransactionId;
- worker->stop_conflict_retention = false;
+ worker->stop_conflict_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1301,27 +1302,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1333,6 +1327,17 @@ ApplyLauncherMain(Datum main_arg)
{
bool updated = false;
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
updated = advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 82d03b5725f..4bb669fa95f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -454,7 +454,6 @@ static void wait_for_publisher_status(RetainConflictInfoData *data,
static void wait_for_local_flush(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,
@@ -4122,10 +4121,6 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
if (!am_leader_apply_worker())
return;
- /* Exit early if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_retention)
- return;
-
switch (data->phase)
{
case RCI_GET_CANDIDATE_XID:
@@ -4288,7 +4283,9 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
static void
wait_for_local_flush(RetainConflictInfoData *data)
{
+ TimestampTz now;
bool stop_conflict_retention = false;
+ bool status_changed;
Assert(!XLogRecPtrIsInvalid(data->remote_lsn) &&
FullTransactionIdIsValid(data->candidate_xid));
@@ -4351,31 +4348,36 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->flushpos_update_time = data->last_recv_time;
}
- /* Check if changes up to the remote_lsn have been applied and flushed */
- if (last_flushpos < data->remote_lsn)
- {
- 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();
- /*
- * 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();
+ /*
+ * If the wait time has exceeded the maximum limit
+ * (max_conflict_retention_duration), stop tracking the non-removable
+ * transaction ID by this apply worker.
+ */
+ if (max_conflict_retention_duration &&
+ TimestampDifferenceExceeds(data->candidate_xid_time, now,
+ max_conflict_retention_duration))
+ stop_conflict_retention = true;
- /*
- * If the wait time has not exceeded the maximum limit
- * (max_conflict_retention_duration), continue waiting for the changes
- * to be applied. Otherwise, stop tracking the non-removable
- * transaction ID by this apply worker.
- */
- if (!max_conflict_retention_duration ||
- !TimestampDifferenceExceeds(data->candidate_xid_time, now,
- max_conflict_retention_duration))
- return;
+ /*
+ * Return if the apply worker is retaining conflict information, and the
+ * changes up to the remote_lsn are not yet applied and flushed.
+ */
+ if (!stop_conflict_retention && last_flushpos < data->remote_lsn)
+ return;
- stop_conflict_retention = true;
- }
+ /*
+ * Determine whether the apply worker is about to stop or resume conflict
+ * retention.
+ */
+ if (MyLogicalRepWorker->stop_conflict_retention != stop_conflict_retention)
+ status_changed = true;
if (!stop_conflict_retention)
{
@@ -4387,13 +4389,25 @@ wait_for_local_flush(RetainConflictInfoData *data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ if (status_changed)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->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));
+
elog(DEBUG2, "confirmed remote flush up to %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(data->remote_lsn),
XidFromFullTransactionId(data->candidate_xid));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
}
- else
+ else if (status_changed)
{
/*
* Reaching here means the time spent applying changes up to the
@@ -4401,9 +4415,9 @@ wait_for_local_flush(RetainConflictInfoData *data)
* (max_conflict_retention_duration). So, we will stop retaining
* conflict information.
*
- * Currently, the retention will not resume automatically unless user
- * manually disable retain_conflict_info and re-enable 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.
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
@@ -4416,6 +4430,13 @@ wait_for_local_flush(RetainConflictInfoData *data)
errdetail("The time spent applying changes up to LSN %X/%X has exceeded the maximum limit of %u ms.",
LSN_FORMAT_ARGS(data->remote_lsn),
max_conflict_retention_duration));
+
+ /*
+ * Notify launcher to either update the xmin of the conflict slot or
+ * invalidate the slot if no other workers are retaining conflict
+ * information.
+ */
+ ApplyLauncherWakeup();
}
/* Notify launcher to update the xmin of the conflict slot */
@@ -4480,51 +4501,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_retention = cause != RS_INVAL_NONE;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-}
-
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4696,16 +4672,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();
}
/*
@@ -5344,8 +5310,6 @@ InitializeLogRepWorker(void)
MySubscription->name)));
CommitTransactionCommand();
-
- update_conflict_retention_status();
}
/*
--
2.30.0.windows.2
v29-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v29-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 5b6ea804d2cd58752ad5f92bc743fdfcfb8541d5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 26 Mar 2025 16:02:56 +0800
Subject: [PATCH v29 6/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 154 ++++++++++++++++++++++-
1 file changed, 152 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 3a4d44e1d0e..1be8383faa5 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 @@ $node_publisher->start;
# 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
@@ -110,4 +110,154 @@ $node_subscriber->wait_for_log(
pass('multiple_unique_conflicts detected during update');
+###############################################################################
+# 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');
+
+# Remember the next transaction ID to be assigned
+my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$node_A->safe_psql(
+ 'postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+$node_B->wait_for_catchup($subname_AB);
+
+# Account for the transaction ID increment caused by enabling the subscription
+$next_xid++;
+
+# 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();
--
2.30.0.windows.2
v29-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v29-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From ff479604292adfd1ac569335d07caa0609a10a40 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 26 Mar 2025 16:18:27 +0800
Subject: [PATCH v29 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 9fcb7c0ff73..9342272fd17 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;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 6636459b870..823d11ecf63 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2205,6 +2205,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5c30ac999ba..498a85804fa 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1381,6 +1381,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 ede89ea3cf9..2c0bb395c00 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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;
@@ -431,6 +445,122 @@ retry:
return found;
}
+/*
+ * 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 f1e92f2fc1a..504ba27209a 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 4bb669fa95f..7ce53f16e23 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2807,17 +2807,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. */
@@ -3155,18 +3166,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 97af7c6554f..9f07b2a4b56 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 8a61ed40f6d..71d42eb06c0 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5647,9 +5647,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 6a1fec88928..c15e04e392c 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"
@@ -783,7 +784,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 6c59125f256..cbd9656a60a 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 e6a2eff3961..82e93d32eb3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2153,6 +2153,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,
@@ -2161,7 +2162,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 1be8383faa5..c3b52d09652 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -114,7 +114,9 @@ pass('multiple_unique_conflicts detected during update');
# 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
@@ -122,7 +124,8 @@ pass('multiple_unique_conflicts detected during update');
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;
@@ -192,6 +195,8 @@ 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.
+# And check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -220,6 +225,15 @@ ok( $stderr =~
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;
+
# Remember the next transaction ID to be assigned
my $next_xid = $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
@@ -227,6 +241,13 @@ $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');
+
# Account for the transaction ID increment caused by enabling the subscription
$next_xid++;
--
2.30.0.windows.2
Dear Hou,
Thanks for updating the patch! I can finally come back to the thread.
Regarding the max_retain_conflict_duration, I prefer GUC approach because it has
already had a mechanism for converting unit: subscription option does not have it.
Below part contains my comments:
01. check_new_cluster_subscription_configuration
```
@@ -2024,6 +2025,7 @@ check_new_cluster_subscription_configuration(void)
PGresult *res;
PGconn *conn;
int max_active_replication_origins;
+ int max_replication_slots;
```
I feel max_replication_slots is needed when old_cluster.sub_retain_conflict_info is true.
02. check_old_cluster_for_valid_slots
```
+ /*
+ * The name "pg_conflict_detection" (defined as
+ * CONFLICT_DETECTION_SLOT) has been reserved for logical
+ * replication conflict detection since PG18.
+ */
+ if (GET_MAJOR_VERSION(new_cluster.major_version) >= 1800 &&
+ strcmp(slot->slotname, "pg_conflict_detection") == 0)
```
IIUC, we can assume that the vesion of new_cluster is same as pg_upgrade, so no
need to check the major version here.
03.
Can we add a test for upgrading subscriber node with retain_conflict_info in 004_subscription?
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Wed, Mar 26, 2025 at 4:17 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here's a rebased version of the patch series.
Thanks Hou-San for the patches. I am going through this long thread
and patches. One doubt I have is whenever there is a chance of
conflict-slot update (either xmin or possibility of its invalidation),
apply-worker gives a wake-up call to the launcher
(ApplyLauncherWakeup). Shouldn't that suffice to wake-up launcher
irrespective of its nap-time? Do we actually need to introduce
MIN/MAX_NAPTIME_PER_SLOT_UPDATE in the launcher and the logic around
it?
thanks
Shveta
On Wed, Apr 16, 2025 at 10:30 AM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Mar 26, 2025 at 4:17 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here's a rebased version of the patch series.
Few comments for patch004:
Config.sgml:
1)
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
IIUC, the above is not entirely accurate. Suppose the subscriber
manages to catch up and sets oldest_nonremovable_xid to 100, which is
then updated in slot. After this, the apply worker takes a nap and
begins a new xid update cycle. Now, let’s say the next candidate_xid
is 200, but this time the subscriber fails to keep up and exceeds
max_conflict_retention_duration. As a result, it sets
oldest_nonremovable_xid to InvalidTransactionId, and the launcher
skips updating the slot’s xmin. However, the previous xmin value (100)
is still there in the slot, causing its data to be retained beyond the
max_conflict_retention_duration. The xid 200 which actually honors
max_conflict_retention_duration was never marked for retention. If my
understanding is correct, then the documentation doesn’t fully capture
this scenario.
2)
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscription, where
Subscription --> subscriptions
3)
Name stop_conflict_retention in MyLogicalRepWorker is confusing. Shall
it be stop_conflict_info_retention?
thanks
Shveta
On Wed, Mar 26, 2025 at 4:17 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here's a rebased version of the patch series.
Thanks for the patches.
While testing the GUC "max_conflict_retention_duration", I noticed a
behavior that seems to bypass its intended purpose.
On Pub, if a txn is stuck in the COMMIT phase for a long time, the
apply_worker on the sub keeps looping in wait_for_publisher_status()
until that Pub's concurrent txn completes its commit.
Due to this, the apply worker can't advance its
oldest_nonremovable_xid and keeps waiting for the Pub's txn to finish.
In such a case, even if the wait time exceeds the configured
max_conflict_retention_duration, conflict retention doesn't stop for
the apply_worker. The conflict info retention is stoppend only once
the Pub's txn is committed and the apply_worker moves to
wait_for_local_flush().
Doesn't this defeat the purpose of max_conflict_retention_duration?
The apply worker has exceeded the max wait time but still retains the
conflict info.
I think we should consider applying the same max time limit check
inside wait_for_publisher_status() as well.
--
Thanks,
Nisha
On Thu, Apr 17, 2025 at 12:19 PM shveta malik wrote:
On Wed, Apr 16, 2025 at 10:30 AM shveta malik <shveta.malik@gmail.com>
wrote:On Wed, Mar 26, 2025 at 4:17 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here's a rebased version of the patch series.
Thanks Hou-San for the patches. I am going through this long thread and
patches. One doubt I have is whenever there is a chance of conflict-slot update
(either xmin or possibility of its invalidation), apply-worker gives a wake-up call
to the launcher (ApplyLauncherWakeup). Shouldn't that suffice to wake-up
launcher irrespective of its nap-time? Do we actually need to introduce
MIN/MAX_NAPTIME_PER_SLOT_UPDATE in the launcher and the logic
around it?
Thanks for reviewing. After rethinking, I agree that the wakeup is
sufficient, so I removed the nap-time logic in this version.
Few comments for patch004: Config.sgml: 1) + <para> + Maximum duration (in milliseconds) for which conflict + information can be retained for conflict detection by the apply worker. + The default value is <literal>0</literal>, indicating that conflict + information is retained until it is no longer needed for detection + purposes. + </para>IIUC, the above is not entirely accurate. Suppose the subscriber manages to
catch up and sets oldest_nonremovable_xid to 100, which is then updated in
slot. After this, the apply worker takes a nap and begins a new xid update cycle.
Now, let’s say the next candidate_xid is 200, but this time the subscriber fails
to keep up and exceeds max_conflict_retention_duration. As a result, it sets
oldest_nonremovable_xid to InvalidTransactionId, and the launcher skips
updating the slot’s xmin.
If the time exceeds the max_conflict_retention_duration, the launcher would
Invalidate the slot, instead of skipping updating it. So the conflict info(e.g.,
dead tuples) would not be retained anymore.
However, the previous xmin value (100) is still there
in the slot, causing its data to be retained beyond the
max_conflict_retention_duration. The xid 200 which actually honors
max_conflict_retention_duration was never marked for retention. If my
understanding is correct, then the documentation doesn’t fully capture this
scenario.
As mentioned above, the strategy here is to invalidate the slot.
2) + The replication slot + <quote><literal>pg_conflict_detection</literal></quote> that used to + retain conflict information will be invalidated if all apply workers + associated with the subscription, whereSubscription --> subscriptions
3)
Name stop_conflict_retention in MyLogicalRepWorker is confusing. Shall it be
stop_conflict_info_retention?
Changed.
Here is V30 patch set includes the following changes:
* Addressed above comments.
* Added the retention timeout check in wait_for_local_flush(), as suggested by Nisha[1]/messages/by-id/CABdArM4Ft8q3dZv4Bqw=rbS5_LFMXDJMRr3vC8a_KMCX1qatpg@mail.gmail.com.
* Improved upgrade codes and added a test for upgrade of retain_conflict_info option,
as suggested by Kuroda-san[2]/messages/by-id/OSCPR01MB14966269726272F2F2B2BD3B0F5B22@OSCPR01MB14966.jpnprd01.prod.outlook.com.
[1]: /messages/by-id/CABdArM4Ft8q3dZv4Bqw=rbS5_LFMXDJMRr3vC8a_KMCX1qatpg@mail.gmail.com
[2]: /messages/by-id/OSCPR01MB14966269726272F2F2B2BD3B0F5B22@OSCPR01MB14966.jpnprd01.prod.outlook.com
Best Regards,
Hou zj
Attachments:
v30-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v30-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From a6b1ae0a08f26f768ee3b7f2685535c88cbbf0fb Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v30 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 2a7a8239a29..3bfe1631934 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 @@ $node_publisher->start;
# 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 @@ $node_subscriber->wait_for_log(
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();
--
2.30.0.windows.2
v30-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v30-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 63b969b83fa9112dc20566c9733dfc605e004579 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 17:33:10 +0800
Subject: [PATCH v30 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 9fcb7c0ff73..9342272fd17 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;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5a9f17beb23..81b202bbebf 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
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c4443249496..d90289f57da 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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 6d595777a87..d2205a84718 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 97af7c6554f..9f07b2a4b56 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 2e7a337762f..864a923bff1 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 ae99407db89..fcd2d1c2336 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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 3bfe1631934..4eacc0c089a 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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 @@ 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.
+# 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 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v30-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v30-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From 9cf79c2fbede0f7638b1a44123e69dc18b4175a2 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v30 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 65bbfc34f50..aab1fc06de2 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;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit
+ phase on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int32</term>
+ <listitem>
+ <para>
+ The epoch of the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2782,6 +2845,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..41fb4fc5025 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 b885513f765..dfc5108da3c 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 10677da56b2..8e42787a426 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 4151a4b2a96..ffbd4e3a02c 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 9fa8beb6103..69dc382d63a 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 e5b945a9ee3..19dd8d7a8b7 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 30b2775952c..b09486017f4 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 86c5f998c77..d554ad39b64 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -122,9 +122,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 ef0b733ebe8..bd8b17a6d0d 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 e5879e00dff..ecf0c2bd701 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
--
2.30.0.windows.2
v30-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v30-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From b72ef318efd7af105fcac6f6fc2acd86a98a7eb5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v30 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 c1674c22cb2..58ce478c493 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4945,6 +4945,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 574a544d9fa..af4028ce2ba 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29711,7 +29711,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29755,7 +29757,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29785,6 +29789,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29807,7 +29813,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index aab1fc06de2..edadb8348ed 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2220,6 +2220,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..eec85cde880 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 8e42787a426..8cef4460848 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 67655111875..85239f6c316 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 ffbd4e3a02c..4922104b018 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 600b87fa9cb..668279cb4e8 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 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
--
2.30.0.windows.2
v30-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v30-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From bde71c8af4228f58179b5966e60c74ffbb086896 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 20:38:20 +0800
Subject: [PATCH v30 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 cbd4e40a320..0fb3894feb6 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8088,6 +8088,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index f288c049a5c..9fcb7c0ff73 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
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index eec85cde880..b7ec696eb3c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,35 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 15efb02badb..637a312e198 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 46d4e65da97..47ba22d28b8 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 8cef4460848..07d686c6d8c 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 4922104b018..471a95da3fb 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 105e917aa7b..0a6b43ccda5 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 b426b5e4736..e9c9aa2f2f9 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -707,6 +707,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 18c2d652bb6..611bc14c1a2 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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 1d08268393e..43a6682a131 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 c916b9299a8..692a7d311cd 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 <name> 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 <name> 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 20fc329992d..0ac7c0b120c 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 1443e1d9292..bff4cc051db 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v30-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v30-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 8fa216d6f79751065b50a3d517ece41f286bf94a Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 20:30:16 +0800
Subject: [PATCH v30 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 58ce478c493..49baa739b62 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5380,6 +5380,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d768ea065c5..5a9f17beb23 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
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index b58c52ea50f..fe552a86fad 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 637a312e198..c4443249496 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 07d686c6d8c..f873a817192 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 @@ retry:
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 471a95da3fb..23e6fb7f221 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 668279cb4e8..bce4af1bdcd 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 60b12446a1c..a5a0d94d32a 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 34826d01380..4cabb0e7c48 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 62beb71da28..2e7a337762f 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 b09486017f4..0fdf49a1938 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v30-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v30-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 7b73ecd4d2f82acc031504f5ccc7ed0ac64193a7 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 19:40:33 +0800
Subject: [PATCH v30 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 49baa739b62..9a9f6fcb218 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5404,7 +5404,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f873a817192..553e5a08772 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -449,7 +449,8 @@ retry:
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->stop_conflict_info_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1287,27 +1288,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1316,6 +1310,17 @@ ApplyLauncherMain(Datum main_arg)
*/
else if (nretain_conflict_info)
{
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 23e6fb7f221..6d595777a87 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -456,7 +456,6 @@ 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,
@@ -4126,10 +4125,6 @@ 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:
@@ -4374,6 +4369,19 @@ wait_for_local_flush(RetainConflictInfoData *data)
if (last_flushpos < data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->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();
}
/*
--
2.30.0.windows.2
On Thu, Apr 24, 2025 at 6:11 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Few comments for patch004: Config.sgml: 1) + <para> + Maximum duration (in milliseconds) for which conflict + information can be retained for conflict detection by the apply worker. + The default value is <literal>0</literal>, indicating that conflict + information is retained until it is no longer needed for detection + purposes. + </para>IIUC, the above is not entirely accurate. Suppose the subscriber manages to
catch up and sets oldest_nonremovable_xid to 100, which is then updated in
slot. After this, the apply worker takes a nap and begins a new xid update cycle.
Now, let’s say the next candidate_xid is 200, but this time the subscriber fails
to keep up and exceeds max_conflict_retention_duration. As a result, it sets
oldest_nonremovable_xid to InvalidTransactionId, and the launcher skips
updating the slot’s xmin.If the time exceeds the max_conflict_retention_duration, the launcher would
Invalidate the slot, instead of skipping updating it. So the conflict info(e.g.,
dead tuples) would not be retained anymore.
launcher will not invalidate the slot until all subscriptions have
stopped conflict_info retention. So info of dead tuples for a
particular oldest_xmin of a particular apply worker could be retained
for much longer than this configured duration. If other apply workers
are actively working (catching up with primary), then they should keep
on advancing xmin of shared slot but if xmin of shared slot remains
same for say 15min+15min+15min for 3 apply-workers (assuming they are
marking themselves with stop_conflict_retention one after other and
xmin of slot has not been advanced), then the first apply worker
having marked itself with stop_conflict_retention still has access to
the oldest_xmin's data for 45 mins instead of 15 mins. (where
max_conflict_retention_duration=15 mins). Please let me know if my
understanding is wrong.
However, the previous xmin value (100) is still there
in the slot, causing its data to be retained beyond the
max_conflict_retention_duration. The xid 200 which actually honors
max_conflict_retention_duration was never marked for retention. If my
understanding is correct, then the documentation doesn’t fully capture this
scenario.As mentioned above, the strategy here is to invalidate the slot.
Please consider the case with multiple subscribers. Sorry if I missed
to mention in my previous email that it was a multi-sub case.
thanks
Shveta
Here is V30 patch set includes the following changes:
Thank You for the patch, please find few concerns:
1)
By looking at code of ApplyLauncherMain(), it appears that we stopped
advancing shared-slot's xmin if any of the subscriptions with
retain_conflict_info is disabled. If a subscription is not being used
and is disabled forever (or for quite long), that means xmin will
never be advanced and we will keep accumulating dead-rows even if
other subscribers with retain_conflict_info are actively setting their
oldest_xmin. This could be problematic. Here too, there should be some
way to set stop-conflict-rettention for such a subscription like we do
for 'subscriber not able to catch-up case'.
But I understand it can be complex to implement as we do not know for
how long a subscription is disabled. If we do not find a simpler way
to implement it, then at least we can document such cases and the
risks associated with disabled subscription which has
'retain_conflict_info' enabled. Thoughts?
2)
in wait_for_local_flush(), we have
should_stop_conflict_info_retention() before 'AllTablesyncsReady'
check. Should we give a discount for table-sync time and avoid doing
stop-conflict-retention when table-sync is going on? This is because
table-sync is one time operation (or done only on
subscription-refresh), so we shall not count time spent in table-sync
for 'max_conflict_retention_duration'. We can reset our timer if
table-sync is observed to be going on. Thoughts?
thanks
Shveta
On Fri, Apr 25, 2025 at 4:05 PM shveta malik <shveta.malik@gmail.com> wrote:
Here is V30 patch set includes the following changes:
Thank You for the patch, please find few concerns:
Please find few more concerns:
3)
In get_candidate_xid(), we first set candidate_xid_time and later
candidate_xid. And between these 2 there are chances that we return
without updating candidate_xid. See 'Return if the
oldest_nonremovable_xid cannot be advanced ' comment. That will leave
'candidate_xid_time' set to new value while 'candidate_xid' is not
yet set.
4)
Do you think there should be some relation between
'xid_advance_interval' and 'max_conflict_retention_duration'? Should
max of 'xid_advance_interval' be limited by
'max_conflict_retention_duration'. Currently say
xid_advance_interval' is set to max 3 mins, what if
'max_conflict_retention_duration' is set to 2 mins? In that case we
will not even check for new xids before 3 mins are over, while
'max_conflict_retention_duration' sets a limit of 2 mins for dead
tuples retention.
thanks
Shveta
On Fri, Apr 25, 2025 at 4:06 PM shveta malik <shveta.malik@gmail.com> wrote:
Here is V30 patch set includes the following changes:
Thank You for the patch, please find few concerns:
1)
By looking at code of ApplyLauncherMain(), it appears that we stopped
advancing shared-slot's xmin if any of the subscriptions with
retain_conflict_info is disabled. If a subscription is not being used
and is disabled forever (or for quite long), that means xmin will
never be advanced and we will keep accumulating dead-rows even if
other subscribers with retain_conflict_info are actively setting their
oldest_xmin. This could be problematic. Here too, there should be some
way to set stop-conflict-rettention for such a subscription like we do
for 'subscriber not able to catch-up case'.
But I understand it can be complex to implement as we do not know for
how long a subscription is disabled. If we do not find a simpler way
to implement it, then at least we can document such cases and the
risks associated with disabled subscription which has
'retain_conflict_info' enabled. Thoughts?
Yeah, I agree that this is the case to be worried about but OTOH, in
such cases, even the corresponding logical_slot on the primary won't
be advanced and lead to bloat on the primary as well. So, we can
probably give WARNING when user tries to disable a subscription or
create a disabled subscription with retention flag true. We may need
to write a LOG or raise a WARNING when while exiting apply worker
disables the subscription due disable_on_error option. In addition to
that, we can even document this case. Will that address your concern?
Few minor comments on 0001:
1.
+ 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"),
Shouldn't this be elog as this is an internal message? And, instead of
"... may be ..", shall we use ".. could be .." in the error message as
the oldest_nonremovable_xid is not yet advanced by this time.
2.
+ * 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.
IIRC, we discussed why it isn't easy to close this race condition. Can
we capture that in comments separately?
--
With Regards,
Amit Kapila.
On Fri, Apr 25, 2025 at 4:06 PM shveta malik <shveta.malik@gmail.com> wrote:
2)
in wait_for_local_flush(), we have
should_stop_conflict_info_retention() before 'AllTablesyncsReady'
check. Should we give a discount for table-sync time and avoid doing
stop-conflict-retention when table-sync is going on? This is because
table-sync is one time operation (or done only on
subscription-refresh), so we shall not count time spent in table-sync
for 'max_conflict_retention_duration'. We can reset our timer if
table-sync is observed to be going on. Thoughts?
Sounds reasonable to me.
3)
In get_candidate_xid(), we first set candidate_xid_time and later
candidate_xid. And between these 2 there are chances that we return
without updating candidate_xid. See 'Return if the
oldest_nonremovable_xid cannot be advanced ' comment. That will leave
'candidate_xid_time' set to new value while 'candidate_xid' is not
yet set.
Good point. I think we should set 'candidate_xid_time' along with
candidate_xid (just after setting candidate_xid).
4)
Do you think there should be some relation between
'xid_advance_interval' and 'max_conflict_retention_duration'? Should
max of 'xid_advance_interval' be limited by
'max_conflict_retention_duration'. Currently say
xid_advance_interval' is set to max 3 mins, what if
'max_conflict_retention_duration' is set to 2 mins? In that case we
will not even check for new xids before 3 mins are over, while
'max_conflict_retention_duration' sets a limit of 2 mins for dead
tuples retention.
Right, ideally, the 'xid_advance_interval' should be set to a value
less than 'max_conflict_retention_duration' when no new_xid is found.
BTW, another related point is that when we decide to stop retaining
dead tuples (via should_stop_conflict_info_retention), should we also
consider the case that the apply worker didn't even try to get the
publisher status because previously it decided that
oldest_nonremovable_xid cannot be advanced due to its
OldestActiveTransactionId?
--
With Regards,
Amit Kapila.
On Thu, May 15, 2025 at 6:02 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Apr 25, 2025 at 4:06 PM shveta malik <shveta.malik@gmail.com> wrote:
Here is V30 patch set includes the following changes:
Thank You for the patch, please find few concerns:
1)
By looking at code of ApplyLauncherMain(), it appears that we stopped
advancing shared-slot's xmin if any of the subscriptions with
retain_conflict_info is disabled. If a subscription is not being used
and is disabled forever (or for quite long), that means xmin will
never be advanced and we will keep accumulating dead-rows even if
other subscribers with retain_conflict_info are actively setting their
oldest_xmin. This could be problematic. Here too, there should be some
way to set stop-conflict-rettention for such a subscription like we do
for 'subscriber not able to catch-up case'.
But I understand it can be complex to implement as we do not know for
how long a subscription is disabled. If we do not find a simpler way
to implement it, then at least we can document such cases and the
risks associated with disabled subscription which has
'retain_conflict_info' enabled. Thoughts?Yeah, I agree that this is the case to be worried about but OTOH, in
such cases, even the corresponding logical_slot on the primary won't
be advanced and lead to bloat on the primary as well. So, we can
probably give WARNING when user tries to disable a subscription or
create a disabled subscription with retention flag true. We may need
to write a LOG or raise a WARNING when while exiting apply worker
disables the subscription due disable_on_error option. In addition to
that, we can even document this case. Will that address your concern?
Yes, it should solve the purpose. Thanks.
thanks
Shveta
On Fri, May 16, 2025 at 11:15 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Apr 25, 2025 at 4:06 PM shveta malik <shveta.malik@gmail.com> wrote:
2)
in wait_for_local_flush(), we have
should_stop_conflict_info_retention() before 'AllTablesyncsReady'
check. Should we give a discount for table-sync time and avoid doing
stop-conflict-retention when table-sync is going on? This is because
table-sync is one time operation (or done only on
subscription-refresh), so we shall not count time spent in table-sync
for 'max_conflict_retention_duration'. We can reset our timer if
table-sync is observed to be going on. Thoughts?Sounds reasonable to me.
3)
In get_candidate_xid(), we first set candidate_xid_time and later
candidate_xid. And between these 2 there are chances that we return
without updating candidate_xid. See 'Return if the
oldest_nonremovable_xid cannot be advanced ' comment. That will leave
'candidate_xid_time' set to new value while 'candidate_xid' is not
yet set.Good point. I think we should set 'candidate_xid_time' along with
candidate_xid (just after setting candidate_xid).4)
Do you think there should be some relation between
'xid_advance_interval' and 'max_conflict_retention_duration'? Should
max of 'xid_advance_interval' be limited by
'max_conflict_retention_duration'. Currently say
xid_advance_interval' is set to max 3 mins, what if
'max_conflict_retention_duration' is set to 2 mins? In that case we
will not even check for new xids before 3 mins are over, while
'max_conflict_retention_duration' sets a limit of 2 mins for dead
tuples retention.Right, ideally, the 'xid_advance_interval' should be set to a value
less than 'max_conflict_retention_duration' when no new_xid is found.BTW, another related point is that when we decide to stop retaining
dead tuples (via should_stop_conflict_info_retention), should we also
consider the case that the apply worker didn't even try to get the
publisher status because previously it decided that
oldest_nonremovable_xid cannot be advanced due to its
OldestActiveTransactionId?
Do you mean avoid stop-conflict-retention in such a case as apply
worker itself did not request status from the publisher? If I
understood your point correctly, then we can do that by advancing the
timer to a new value even if we did not update candidate-xid and did
not ask the status from the publisher. I think it is already happening
in get_candidate_xid(). It updates the timer but not the xid (my
concern #3 can be ignored then).
thanks
Shveta
On Fri, Apr 25, 2025 at 10:08 AM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Apr 24, 2025 at 6:11 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Few comments for patch004: Config.sgml: 1) + <para> + Maximum duration (in milliseconds) for which conflict + information can be retained for conflict detection by the apply worker. + The default value is <literal>0</literal>, indicating that conflict + information is retained until it is no longer needed for detection + purposes. + </para>IIUC, the above is not entirely accurate. Suppose the subscriber manages to
catch up and sets oldest_nonremovable_xid to 100, which is then updated in
slot. After this, the apply worker takes a nap and begins a new xid update cycle.
Now, let’s say the next candidate_xid is 200, but this time the subscriber fails
to keep up and exceeds max_conflict_retention_duration. As a result, it sets
oldest_nonremovable_xid to InvalidTransactionId, and the launcher skips
updating the slot’s xmin.If the time exceeds the max_conflict_retention_duration, the launcher would
Invalidate the slot, instead of skipping updating it. So the conflict info(e.g.,
dead tuples) would not be retained anymore.launcher will not invalidate the slot until all subscriptions have
stopped conflict_info retention. So info of dead tuples for a
particular oldest_xmin of a particular apply worker could be retained
for much longer than this configured duration. If other apply workers
are actively working (catching up with primary), then they should keep
on advancing xmin of shared slot but if xmin of shared slot remains
same for say 15min+15min+15min for 3 apply-workers (assuming they are
marking themselves with stop_conflict_retention one after other and
xmin of slot has not been advanced), then the first apply worker
having marked itself with stop_conflict_retention still has access to
the oldest_xmin's data for 45 mins instead of 15 mins. (where
max_conflict_retention_duration=15 mins). Please let me know if my
understanding is wrong.
IIUC, the current code will stop updating the slot even if one of the
apply workers has set stop_conflict_info_retention. The other apply
workers will keep on maintaining their oldest_nonremovable_xid without
advancing the slot. If this is correct, then what behavior instead we
expect here? Do we want the slot to keep advancing till any worker is
actively maintaining oldest_nonremovable_xid? To some extent, this
matches with the cases where the user has set retain_conflict_info for
some subscriptions but not for others.
If so, how will users eventually know for which tables they can expect
to reliably detect update_delete? One possibility is that users can
check which apply workers have stopped maintaining
oldest_nonremovable_xid via pg_stat_subscription view and then see the
tables corresponding to those subscriptions. Also, what will we do as
part of the resolutions in the applyworkers where
stop_conflict_info_retention is set? Shall we simply LOG that we can't
resolve this conflict and continue till the user takes some action, or
simply error out in such cases?
--
With Regards,
Amit Kapila.
On Fri, May 16, 2025 at 12:01 PM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, May 16, 2025 at 11:15 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
BTW, another related point is that when we decide to stop retaining
dead tuples (via should_stop_conflict_info_retention), should we also
consider the case that the apply worker didn't even try to get the
publisher status because previously it decided that
oldest_nonremovable_xid cannot be advanced due to its
OldestActiveTransactionId?Do you mean avoid stop-conflict-retention in such a case as apply
worker itself did not request status from the publisher? If I
understood your point correctly, then we can do that by advancing the
timer to a new value even if we did not update candidate-xid and did
not ask the status from the publisher.
But candidate_xid_time is also used in wait_for_local_flush() to check
clock_skew between publisher and subscriber, so for that purpose, it
is better to set it along with candidate_xid. However, can't we rely
on the valid value of candidate_xid to ensure that apply worker didn't
send any request? Note that we always reset candidate_xid once we have
updated oldest_nonremovable_xid.
--
With Regards,
Amit Kapila.
On Fri, May 16, 2025 at 12:17 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Apr 25, 2025 at 10:08 AM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Apr 24, 2025 at 6:11 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Few comments for patch004: Config.sgml: 1) + <para> + Maximum duration (in milliseconds) for which conflict + information can be retained for conflict detection by the apply worker. + The default value is <literal>0</literal>, indicating that conflict + information is retained until it is no longer needed for detection + purposes. + </para>IIUC, the above is not entirely accurate. Suppose the subscriber manages to
catch up and sets oldest_nonremovable_xid to 100, which is then updated in
slot. After this, the apply worker takes a nap and begins a new xid update cycle.
Now, let’s say the next candidate_xid is 200, but this time the subscriber fails
to keep up and exceeds max_conflict_retention_duration. As a result, it sets
oldest_nonremovable_xid to InvalidTransactionId, and the launcher skips
updating the slot’s xmin.If the time exceeds the max_conflict_retention_duration, the launcher would
Invalidate the slot, instead of skipping updating it. So the conflict info(e.g.,
dead tuples) would not be retained anymore.launcher will not invalidate the slot until all subscriptions have
stopped conflict_info retention. So info of dead tuples for a
particular oldest_xmin of a particular apply worker could be retained
for much longer than this configured duration. If other apply workers
are actively working (catching up with primary), then they should keep
on advancing xmin of shared slot but if xmin of shared slot remains
same for say 15min+15min+15min for 3 apply-workers (assuming they are
marking themselves with stop_conflict_retention one after other and
xmin of slot has not been advanced), then the first apply worker
having marked itself with stop_conflict_retention still has access to
the oldest_xmin's data for 45 mins instead of 15 mins. (where
max_conflict_retention_duration=15 mins). Please let me know if my
understanding is wrong.IIUC, the current code will stop updating the slot even if one of the
apply workers has set stop_conflict_info_retention. The other apply
workers will keep on maintaining their oldest_nonremovable_xid without
advancing the slot. If this is correct, then what behavior instead we
expect here?
I think this is not the current behaviour.
Do we want the slot to keep advancing till any worker is
actively maintaining oldest_nonremovable_xid?
In fact, this one is the current behaviour of v30 patch.
To some extent, this
matches with the cases where the user has set retain_conflict_info for
some subscriptions but not for others.If so, how will users eventually know for which tables they can expect
to reliably detect update_delete? One possibility is that users can
check which apply workers have stopped maintaining
oldest_nonremovable_xid via pg_stat_subscription view and then see the
tables corresponding to those subscriptions.
Yes, it is a possibility, but I feel it will be too much to monitor
from the user's perspective.
Also, what will we do as
part of the resolutions in the applyworkers where
stop_conflict_info_retention is set? Shall we simply LOG that we can't
resolve this conflict and continue till the user takes some action, or
simply error out in such cases?
We can LOG. Erroring out again will prevent the subscriber from
proceeding, and the subscriber initially reached this state due to
falling behind, which led to stop_conflict_retention=true. But still
if we go with erroring out, I am not very sure what action users can
take in this situation? Subscriber is still lagging and if the user
recreates the slot as a solution, apply worker will soon go to
'stop_conflict_retention=true' state again, provided the subscriber is
still not able to catch-up.
thanks
Shveta
On Fri, May 16, 2025 at 2:40 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, May 16, 2025 at 12:01 PM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, May 16, 2025 at 11:15 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
BTW, another related point is that when we decide to stop retaining
dead tuples (via should_stop_conflict_info_retention), should we also
consider the case that the apply worker didn't even try to get the
publisher status because previously it decided that
oldest_nonremovable_xid cannot be advanced due to its
OldestActiveTransactionId?Do you mean avoid stop-conflict-retention in such a case as apply
worker itself did not request status from the publisher? If I
understood your point correctly, then we can do that by advancing the
timer to a new value even if we did not update candidate-xid and did
not ask the status from the publisher.But candidate_xid_time is also used in wait_for_local_flush() to check
clock_skew between publisher and subscriber, so for that purpose, it
is better to set it along with candidate_xid. However, can't we rely
on the valid value of candidate_xid to ensure that apply worker didn't
send any request? Note that we always reset candidate_xid once we have
updated oldest_nonremovable_xid.
I think this is automatically taken care of because we call
should_stop_conflict_info_retention() only during 'wait' phase, which
should be done after candidate_xid is set. Having said that, we should
have assert for candidate_xid in should_stop_conflict_info_retention()
and also add in comments that it should be called only during the
'wait' phase. Additionally, we can also have an assert that
should_stop_conflict_info_retention() is called only during the 'wait'
phase.
--
With Regards,
Amit Kapila.
On Thu, May 15, 2025 at 6:02 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Few minor comments on 0001: 1. + 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"),Shouldn't this be elog as this is an internal message? And, instead of
"... may be ..", shall we use ".. could be .." in the error message as
the oldest_nonremovable_xid is not yet advanced by this time.2. + * 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.IIRC, we discussed why it isn't easy to close this race condition. Can
we capture that in comments separately?
A few more comments:
=================
3.
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
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return;
/* Exit early if we have already stopped retaining */
if (MyLogicalRepWorker->stop_conflict_info_retention)
return;
...
get_candidate_xid()
{
...
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
data->xid_advance_interval))
return;
Would it be better to encapsulate all these preliminary checks that
decide whether we can move to computing oldest_nonremovable_xid in a
separate function? The check in get_candidate_xid would require some
additional conditions because it is not required in every phase.
Additionally, we can move the core phase processing logic to compute
in a separate function. We can try this to see if the code looks
better with such a refactoring.
4.
+ /*
+ * 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;
I think there is a possibility of optimization here for cases where
there are no new transactions on the publisher side across the next
cycle of advancement of oldest_nonremovable_xid. We can simply set
candidate_xid as oldest_nonremovable_xid instead of going into
RCI_WAIT_FOR_LOCAL_FLUSH phase. If you want to keep the code simple
for the first version, then at least note that down in comments, but
OTOH, if it is simple to achieve, then let's do it now.
--
With Regards,
Amit Kapila.
Please find few more comments:
1)
ProcessStandbyPSRequestMessage:
+ /*
+ * 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");
a) This error is not clear. Is it supposed to be user oriented error
or internal error? As a user, it is difficult to interpret this error
and take some action.
b) What I understood is that there is no user of enabling
'retain_conflict_info' for a subscription which is subscribing to a
publisher which is physical standby too. So shall we emit such an
ERROR during 'Create Sub(retain_conflict_info=on)' itself? But that
would need checking whether the publisher is physical standby or not
during CREATE-SUB. Is that a possibility?
2)
----------
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
----------
Why do we need this call to 'maybe_advance_nonremovable_xid' towards
end of LogicalRepApplyLoop() i.e. the last call? Can it make any
further 'data.phase' change here? IIUC, there are 2 triggers for
'data.phase' change through LogicalRepApplyLoop(). First one is for
the very first time when we start this loop, it will set data.phase to
0 (RCI_GET_CANDIDATE_XID) and will call
maybe_advance_nonremovable_xid(). After that, LogicalRepApplyLoop()
function can trigger a 'data.phase' change only when it receives a
response from the publisher. Shouldn't the first 4 calls
to maybe_advance_nonremovable_xid() from LogicalRepApplyLoop() suffice?
3)
Code is almost the same for GetOldestActiveTransactionId() and
GetOldestTransactionIdInCommit(). I think we can merge these two.
GetOldestActiveTransactionId() can take new arg "getTxnInCommit".
GetOldestTransactionIdInCommit() can call
GetOldestActiveTransactionId() with that arg as true, whereas other 2
callers can pass it as false.
thanks
Shveta
On Tue, May 20, 2025 at 8:38 AM shveta malik <shveta.malik@gmail.com> wrote:
Please find few more comments:
1) ProcessStandbyPSRequestMessage: + /* + * 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");a) This error is not clear. Is it supposed to be user oriented error
or internal error? As a user, it is difficult to interpret this error
and take some action.
This is an internal error for developers to understand that they have
sent the wrong message. Do you have any suggestions to change it?
b) What I understood is that there is no user of enabling
'retain_conflict_info' for a subscription which is subscribing to a
publisher which is physical standby too. So shall we emit such an
ERROR during 'Create Sub(retain_conflict_info=on)' itself? But that
would need checking whether the publisher is physical standby or not
during CREATE-SUB. Is that a possibility?
The 0003 patch already took care of this, see check_conflict_info_retaintion.
--
With Regards,
Amit Kapila.
On Tue, May 20, 2025 at 10:24 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, May 20, 2025 at 8:38 AM shveta malik <shveta.malik@gmail.com> wrote:
Please find few more comments:
1) ProcessStandbyPSRequestMessage: + /* + * 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");a) This error is not clear. Is it supposed to be user oriented error
or internal error? As a user, it is difficult to interpret this error
and take some action.This is an internal error for developers to understand that they have
sent the wrong message. Do you have any suggestions to change it?
The current message is fine if point b) is already taken care of.
b) What I understood is that there is no user of enabling
'retain_conflict_info' for a subscription which is subscribing to a
publisher which is physical standby too. So shall we emit such an
ERROR during 'Create Sub(retain_conflict_info=on)' itself? But that
would need checking whether the publisher is physical standby or not
during CREATE-SUB. Is that a possibility?The 0003 patch already took care of this, see check_conflict_info_retaintion.
Okay, thanks. Missed it somehow during review.
thanks
Shveta
On Tue, May 20, 2025 at 8:38 AM shveta malik <shveta.malik@gmail.com> wrote:
Please find few more comments:
1) ProcessStandbyPSRequestMessage: + /* + * 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");a) This error is not clear. Is it supposed to be user oriented error
or internal error? As a user, it is difficult to interpret this error
and take some action.b) What I understood is that there is no user of enabling
'retain_conflict_info' for a subscription which is subscribing to a
publisher which is physical standby too. So shall we emit such an
ERROR during 'Create Sub(retain_conflict_info=on)' itself? But that
would need checking whether the publisher is physical standby or not
during CREATE-SUB. Is that a possibility?2)
----------
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 ----------Why do we need this call to 'maybe_advance_nonremovable_xid' towards
end of LogicalRepApplyLoop() i.e. the last call? Can it make any
further 'data.phase' change here? IIUC, there are 2 triggers for
'data.phase' change through LogicalRepApplyLoop(). First one is for
the very first time when we start this loop, it will set data.phase to
0 (RCI_GET_CANDIDATE_XID) and will call
maybe_advance_nonremovable_xid(). After that, LogicalRepApplyLoop()
function can trigger a 'data.phase' change only when it receives a
response from the publisher. Shouldn't the first 4 calls
to maybe_advance_nonremovable_xid() from LogicalRepApplyLoop() suffice?3)
Code is almost the same for GetOldestActiveTransactionId() and
GetOldestTransactionIdInCommit(). I think we can merge these two.
GetOldestActiveTransactionId() can take new arg "getTxnInCommit".
GetOldestTransactionIdInCommit() can call
GetOldestActiveTransactionId() with that arg as true, whereas other 2
callers can pass it as false.
Few more comments mostly for patch001:
4)
For this feature, since we are only interested in remote UPDATEs
happening concurrently, so shall we ask primary to provide oldest
"UPDATE" transaction-id in commit-phase rather than any operation's
transaction-id? This may avoid unnecessarily waiting and pinging at
subscriber's end in order to advance oldest_nonremovable-xid.
Thoughts?
5)
+
+/*
+ * GetOldestTransactionIdInCommit()
+ *
+ * Similar to GetOldestActiveTransactionId but returns the oldest
transaction ID
+ * that is currently in the commit phase.
+ */
+TransactionId
+GetOldestTransactionIdInCommit(void)
If there is no transaction currently in 'commit' phase, this function
will then return the next transaction-id. Please mention this in the
comments. I think the doc 'protocol-replication.html' should also be
updated for the same.
6)
+ data->flushpos_update_time = 0;
Do we really need to reset this 'flushpos_update_time' at the end of
wait_for_local_flush()? Even in the next cycle (when the phase
restarts from RCI_GET_CANDIDATE_XID), we can reuse this time to decide
whether we should call get_flush_position() again or skip it, when in
wait_for_local_flush().
7)
+/*
+ * 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;
I think we record/update last_flushpos in wait_for_local_flush() as
well. Shall we update comments accordingly?
8)
Shall we rename "check_conflict_info_retaintion" to
"check_conflict_info_retention" or
"check_remote_for_retainconflictinfo"? ('retaintion' is not a correct
word)
9)
In RetainConflictInfoData, we can keep reply_time along with other
remote_* variables. The idea is to separate the variables received in
remote's response from the ones purely set and reset by the local
node.
thanks
Shveta
On Fri, May 16, 2025 at 5:01 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Please find some more comments on the 0001 patch:
1.
We need to know about such transactions
+ * for conflict detection and resolution in logical replication. See
+ * GetOldestTransactionIdInCommit and its use.
Do we need to mention resolution in the above sentence? This patch is
all about detecting conflict reliably.
2. In wait_for_publisher_status(), we use remote_epoch,
remote_nextxid, and remote_oldestxid to compute full transaction id's.
Why can't we send FullTransactionIds for remote_oldestxid and
remote_nextxid from publisher? If these are required, maybe a comment
somewhere for that would be good.
3.
/*
+ * 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();
How does setting committs after setting DELAY_CHKPT_IN_COMMIT help in
advancing client-side xid? IIUC, on client-side, we simply wait for
such an xid to be finished based on the remote_oldestxid and
remote_nextxid sent via the server. So, the above comment is not
completely clear to me. I have updated this and related comments in
the attached diff patch to make it clear. See if that makes sense to
you.
4.
In 0001's commit message, we have: "Furthermore, the preserved commit
timestamps and origin data are essential for
consistently detecting update_origin_differs conflicts." But it is not
clarified how this patch helps in consistently detecting
update_origin_differs conflict?
5. I have tried to add some more details in comments on why
oldest_nonremovable_xid needs to be FullTransactionId. See attached.
--
With Regards,
Amit Kapila.
Attachments:
v30_0001_amit.1.patch.txttext/plain; charset=US-ASCII; name=v30_0001_amit.1.patch.txtDownload
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 41fb4fc5025..ea508542745 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2329,9 +2329,9 @@ RecordTransactionCommitPrepared(TransactionId xid,
/*
* 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
+ * we want to ensure all transactions that have acquired commit timestamp
+ * 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();
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index dfc5108da3c..6e1dc76744f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,6 +1431,10 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
Assert((MyProc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0);
START_CRIT_SECTION();
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0fdf49a1938..7d1e2096232 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -99,7 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ * could falsely make this ID appear as if it originates from the future
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
*/
FullTransactionId oldest_nonremovable_xid;
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 65ecf3280fb..c6f5ebceefd 100644
--- a/src/include/storage/proc.h
+++ b/src/include/storage/proc.h
@@ -133,10 +133,10 @@ extern PGDLLIMPORT int FastPathLockGroupsPerBackend;
*
* 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.
+ * to a transaction being within a critical commit section. We need this new
+ * flag to ensure all the transactions that have acquired commit timestamp 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)
On Tue, May 20, 2025 at 7:41 PM Amit Kapila wrote:
On Fri, May 16, 2025 at 5:0 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:Please find some more comments on the 0001 patch:
Thanks for the comments!
1. We need to know about such transactions + * for conflict detection and resolution in logical replication. See + * GetOldestTransactionIdInCommit and its use.Do we need to mention resolution in the above sentence? This patch is
all about detecting conflict reliably.
I think it’s not needed. Removed.
2. In wait_for_publisher_status(), we use remote_epoch,
remote_nextxid, and remote_oldestxid to compute full transaction id's.
Why can't we send FullTransactionIds for remote_oldestxid and
remote_nextxid from publisher? If these are required, maybe a comment
somewhere for that would be good.
It was intended to follow the existing "hot standby feedback" message, but
since in our case we do not need to use the epoch and xid separately, so I
changed it to FullTransactionId in this version.
3. /* + * 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();How does setting committs after setting DELAY_CHKPT_IN_COMMIT help in
advancing client-side xid? IIUC, on client-side, we simply wait for
such an xid to be finished based on the remote_oldestxid and
remote_nextxid sent via the server. So, the above comment is not
completely clear to me. I have updated this and related comments in
the attached diff patch to make it clear. See if that makes sense to you.
It looks good to me. Merged.
4.
In 0001's commit message, we have: "Furthermore, the preserved commit
timestamps and origin data are essential for consistently detecting
update_origin_differs conflicts." But it is not clarified how this
patch helps in consistently detecting update_origin_differs conflict?
The reason is the replication slot can also prevent the origin data from being
removed. The origin data could be removed by vacuum freeze, which could also be
blocked due to the replication slot.xmin. I have added it the commit message.
Attaching the V31 patch set which addressed comments in [1]/messages/by-id/CAJpy0uD6SgD7w839Wzezdj0JT2OnA+xCxddM15=gb5nRqYAv+A@mail.gmail.com~[8]/messages/by-id/CAA4eK1+=ZAf0T2iMg2+ZF4cJdUk=UViqpiOg_kPa8jgK+g94aw@mail.gmail.com.
The comments in [9]/messages/by-id/CAA4eK1LLaXzsKOaPwGTiikOYySYK+Ty_x3EXg-g=7M_CLn4WiQ@mail.gmail.com concerning the new GUC in patch 0004 is still under review
and will be addressed in the next version.
[1]: /messages/by-id/CAJpy0uD6SgD7w839Wzezdj0JT2OnA+xCxddM15=gb5nRqYAv+A@mail.gmail.com
[2]: /messages/by-id/CAJpy0uCYqG16zCjiCK4og6yqR7zP2rB1oOT7=AnDdVePo-8RrA@mail.gmail.com
[3]: /messages/by-id/CAA4eK1KemsW0EXaSy2Y-M-vVy5Gh4onNG++kKs7ugY+3N-g-Yw@mail.gmail.com
[4]: /messages/by-id/CAA4eK1+r9V6DpH9gYRa2xOx167FapbuKdc4gESr8Etxpx2zrqw@mail.gmail.com
[5]: /messages/by-id/CAJpy0uArh0A9yOxoamD0RWM-7K9kyoUMNh5C2+PFTbGFoxf5wg@mail.gmail.com
[6]: /messages/by-id/CAJpy0uDL4oLdhYup44a2=1OeyUSsKhg8Y30-h1uxcf=mki57uA@mail.gmail.com
[7]: /messages/by-id/CAA4eK1+VNaGi-GU6awgFKmTgidLTHo2HDuzV1+aT8sjn8QtPxg@mail.gmail.com
[8]: /messages/by-id/CAA4eK1+=ZAf0T2iMg2+ZF4cJdUk=UViqpiOg_kPa8jgK+g94aw@mail.gmail.com
[9]: /messages/by-id/CAA4eK1LLaXzsKOaPwGTiikOYySYK+Ty_x3EXg-g=7M_CLn4WiQ@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v31-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v31-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From cde95f88afc3652b2e6eafd1615c6edd6caec70b Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Wed, 21 May 2025 20:00:02 +0800
Subject: [PATCH v31 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 | 35 ++++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 155 ++++++++++++++++--
src/backend/replication/logical/launcher.c | 51 ++++--
src/backend/replication/logical/worker.c | 68 +++++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 4 +
src/test/regress/expected/subscription.out | 179 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
22 files changed, 579 insertions(+), 131 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cbd4e40a320..0fb3894feb6 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8088,6 +8088,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index f288c049a5c..9fcb7c0ff73 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
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index eec85cde880..739161df715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,41 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 15efb02badb..637a312e198 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 46d4e65da97..8836538076e 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_pub_conflict_info_retention(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,10 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ opts.enabled);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1110,6 +1135,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 +1191,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 +1351,27 @@ 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;
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ sub->enabled);
+
+ /*
+ * 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] =
@@ -1347,6 +1394,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Skip the track_commit_timestamp check by passing it as true,
+ * since it has already been validated during CREATE
+ * SUBSCRIPTION and ALTER SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ true, opts.enabled);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1410,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 +1432,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 +1638,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 +1655,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 +1671,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_pub_conflict_info_retention(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 +2274,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_pub_conflict_info_retention(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 8cef4460848..07d686c6d8c 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 801ed8d0adf..9def348a05d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -446,6 +446,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *data);
static void process_rci_phase_transition(RetainConflictInfoData *data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *data);
@@ -4115,6 +4116,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(data))
+ return;
+
+ process_rci_phase_transition(data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4122,9 +4136,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -4535,6 +4553,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 ||
@@ -4544,7 +4571,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,
@@ -5364,6 +5392,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check by passing it as true, since it
+ * has already been validated during CREATE SUBSCRIPTION and ALTER
+ * SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true,
+ false);
+
proc_exit(0);
}
@@ -5718,3 +5754,29 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Give a warning if track_commit_timestamp is not enabled or if the
+ * subscription is disabled.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool track_commit_timestamp,
+ bool sub_enabled)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (!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 (!sub_enabled)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("information for detecting conflicts cannot be purged when the subscription is disabled"));
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index c73e73a87d1..1d68453ae6e 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 7417eab6aef..945b3fce670 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 940fc77fc2e..480dc126a5f 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();
}
@@ -2017,9 +2017,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)
@@ -2027,6 +2028,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)
@@ -2052,6 +2055,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);
@@ -2114,6 +2142,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 1d08268393e..43a6682a131 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 ec65ab79fec..cbf2703c190 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3728,8 +3729,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ae46fa50ce6 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,8 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool track_commit_timestamp,
+ bool sub_enabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..310eb362511 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,45 @@ 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: information for detecting conflicts cannot be purged when the subscription 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v31-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v31-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 2917377c69eb5b1703c8d9b8a6bebfb72f1e1398 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Wed, 21 May 2025 19:48:51 +0800
Subject: [PATCH v31 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 | 186 ++++++++++++++++--
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, 357 insertions(+), 26 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 291fbfc5319..77ee536b258 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5379,6 +5379,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index b58c52ea50f..fe552a86fad 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 637a312e198..c4443249496 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 07d686c6d8c..f873a817192 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 @@ retry:
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 9def348a05d..5a90cc0dfa8 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"
@@ -454,8 +455,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,
@@ -3866,7 +3870,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,
@@ -4142,6 +4147,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4284,6 +4293,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;
+
if (!FullTransactionIdIsValid(data->last_phase_at))
data->last_phase_at = data->remote_nextxid;
@@ -4352,6 +4368,25 @@ wait_for_local_flush(RetainConflictInfoData *data)
* have a WAL position greater than the data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync is
+ * an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ data->candidate_xid_time = data->last_recv_time
+ ? data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(data))
return;
/*
@@ -4377,14 +4412,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;
@@ -4397,10 +4436,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 */
+ process_rci_phase_transition(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 = InvalidFullTransactionId;
@@ -4408,9 +4456,64 @@ wait_for_local_flush(RetainConflictInfoData *data)
data->reply_time = 0;
data->last_phase_at = InvalidFullTransactionId;
data->candidate_xid = InvalidFullTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(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;
+
+ Assert(FullTransactionIdIsValid(data->candidate_xid));
+ Assert(data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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;
}
/*
@@ -4443,6 +4546,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *data, bool new_xid_found)
*/
data->xid_advance_interval = Min(data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ data->xid_advance_interval = Min(data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
@@ -4454,6 +4561,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.
*/
@@ -4625,6 +4777,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();
}
/*
@@ -5271,6 +5433,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 668279cb4e8..bce4af1bdcd 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 2f8cbd86759..e8249ba8491 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 34826d01380..4cabb0e7c48 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 62beb71da28..2e7a337762f 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 849cd3ed765..7f9b9aa26dd 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -121,6 +121,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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v31-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v31-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 7b683e8e4a581907c5fa53118bd77cac07f9988c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 17:33:10 +0800
Subject: [PATCH v31 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 9fcb7c0ff73..9342272fd17 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;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will arise.
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c4443249496..d90289f57da 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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 c3b98f387d0..88bf054e45a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2814,17 +2814,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. */
@@ -3162,18 +3173,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 97af7c6554f..9f07b2a4b56 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 2e7a337762f..864a923bff1 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 ae99407db89..fcd2d1c2336 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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 3bfe1631934..4eacc0c089a 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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 @@ 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.
+# 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 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v31-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchapplication/octet-stream; name=v31-0001-Maintain-the-oldest-non-removeable-tranasction-I.patchDownload
From d582e94dcc0751d2b8000636d6f5911387e77883 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v31 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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these 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 | 86 ++++
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 10 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 508 ++++++++++++++++++++-
src/backend/replication/walsender.c | 56 +++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/worker_internal.h | 35 ++
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
12 files changed, 734 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..eadbd49e696 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2643,6 +2643,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID along that is currently in the commit
+ phase on the server, along with its epoch. The most significant 32
+ bits are the epoch. The least significant 32 bits are the
+ transaction ID. If no transactions are active on the server, this
+ number will be the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2846,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..b0ddc25c852 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 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..6e1dc76744f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,14 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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 +1541,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1914859b2ee..5f19981f9c3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 10677da56b2..8e42787a426 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 4151a4b2a96..c09e37490b4 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in the
+ * commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned on
+ * the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ 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 */
+
+ /*
+ * The following fields are used to determine the timing for the next round
+ * of transaction ID advancement and the update of the remote flush
+ * position.
+ */
+ TimestampTz last_recv_time; /* when the last message was received */
+ TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
+ TimestampTz flushpos_update_time; /* when the remote flush position was
+ * updated in final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ 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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +444,18 @@ 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 process_rci_phase_transition(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 +3661,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 +3740,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 +3768,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 +3785,31 @@ 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 = FullTransactionIdFromU64(pq_getmsgint64(&s));
+ data.remote_nextxid = FullTransactionIdFromU64(pq_getmsgint64(&s));
+ 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 +3822,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 +3861,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 +3931,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 +3968,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 +4045,394 @@ 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.
+ *
+ * XXX In phase RCI_REQUEST_PUBLISHER_STATUS, a potential enhancement could be
+ * requesting transaction information specifically for those containing
+ * UPDATEs. However, this approach introduces additional complexities in
+ * tracking UPDATEs for transactions on the publisher, and it may not
+ * effectively address scenarios with frequent UPDATEs.
+ */
+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;
+
+ process_rci_phase_transition(data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *data,
+ bool status_received)
+{
+ 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;
+
+ /*
+ * Immediately update the timer, even if the function returns later without
+ * setting candidate_xid due to inactivity on the subscriber. This ensures
+ * a certain interval before recalculating candidate_xid, minimizing
+ * frequent calls to GetOldestActiveTransactionId.
+ */
+ data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+ 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 */
+ process_rci_phase_transition(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)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(data->last_phase_at))
+ data->last_phase_at = data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * Directly advancing the non-removable transaction ID is possible if there
+ * are no activities on the publisher since the last advancement cycle.
+ * However, it requires maintaining two fields, last_remote_nextxid and
+ * last_remote_lsn, within the structure for comparison with the current
+ * cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opt not to
+ * advance transaction ID in this case.
+ */
+ if (FullTransactionIdPrecedesOrEquals(data->last_phase_at,
+ data->remote_oldestxid))
+ data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(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,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 = InvalidFullTransactionId;
+ data->remote_nextxid = InvalidFullTransactionId;
+ data->reply_time = 0;
+ data->last_phase_at = InvalidFullTransactionId;
+ data->candidate_xid = InvalidFullTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(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 9fa8beb6103..d3e2bf68d73 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, U64FromFullTransactionId(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 e5b945a9ee3..c1d5ddbad4b 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * If inCommitOnly is true, we target transactions currently in the commit
+ * phase only.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..849cd3ed765 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,41 @@ 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing the above race condition isn't straightforward. One approach
+ * could be making the apply worker wait for slot creation before updating
+ * oldest_nonremovable_xid. However, creating a race-free mechanism is
+ * difficult, especially when users toggle retain_conflict_info on and off
+ * concurrently. It is crucial to ensure that after disabling
+ * retain_conflict_info and dropping the slot, the launcher doesn't
+ * prematurely recreate it if a subscription re-enables the option before
+ * workers reset their oldest_nonremovable_xid.
+ *
+ * One might also consider managing slot creation and deletion within
+ * subscription DDLs using strong locks to avoid race conditions. Yet, this
+ * method is not entirely reliable because commands can be rolled back;
+ * even if slot creation is deferred to the final step after database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
+ */
+ 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 9ea573fae21..00d9cc54c71 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2558,6 +2558,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v31-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v31-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 4ec97dd7f1002bbfc4a503de5c79a849ecac26ff Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Sep 2024 12:11:34 +0800
Subject: [PATCH v31 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 23d2b1be424..291fbfc5319 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4945,6 +4945,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index b405525a465..4d98bf9bf19 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29710,7 +29710,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29754,7 +29756,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29784,6 +29788,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29806,7 +29812,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index eadbd49e696..7a40c62c59b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..eec85cde880 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 8e42787a426..8cef4460848 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 67655111875..85239f6c316 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 c09e37490b4..801ed8d0adf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4376,6 +4376,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 600b87fa9cb..668279cb4e8 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 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
--
2.30.0.windows.2
v31-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v31-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 540a3b0a53754cff49d6ad4b027b1a905e8a28d2 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Wed, 21 May 2025 19:50:13 +0800
Subject: [PATCH v31 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 77ee536b258..e25cffe973d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5403,7 +5403,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f873a817192..553e5a08772 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -449,7 +449,8 @@ retry:
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->stop_conflict_info_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1287,27 +1288,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1316,6 +1310,17 @@ ApplyLauncherMain(Datum main_arg)
*/
else if (nretain_conflict_info)
{
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5a90cc0dfa8..c3b98f387d0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -459,7 +459,6 @@ 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,
@@ -4147,10 +4146,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4419,6 +4414,19 @@ wait_for_local_flush(RetainConflictInfoData *data)
if (last_flushpos < data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->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
@@ -4427,6 +4435,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",
@@ -4466,9 +4475,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)
@@ -4497,19 +4505,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);
@@ -4561,51 +4576,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.
*/
@@ -4777,16 +4747,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();
}
/*
@@ -5433,8 +5393,6 @@ InitializeLogRepWorker(void)
MySubscription->name)));
CommitTransactionCommand();
-
- update_conflict_retention_status();
}
/*
--
2.30.0.windows.2
v31-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v31-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 6945aabb9f3da41127142d46ff41bb9b8e0e11d7 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v31 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 2a7a8239a29..3bfe1631934 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 @@ $node_publisher->start;
# 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 @@ $node_subscriber->wait_for_log(
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();
--
2.30.0.windows.2
On Tue, May 20, 2025 at 6:30 PM shveta malik wrote:
Few more comments mostly for patch001:
Thanks for the comments!
4)
For this feature, since we are only interested in remote UPDATEs happening
concurrently, so shall we ask primary to provide oldest "UPDATE"
transaction-id in commit-phase rather than any operation's transaction-id?
This may avoid unnecessarily waiting and pinging at subscriber's end in order
to advance oldest_nonremovable-xid.
Thoughts?
It is possible, but considering the potential complexity/cost to track UPDATE
operations in top-level and sub-transactions, coupled with its limited benefit
for workloads featuring frequent UPDATEs on publishers such as observed during
TPC-B performance tests, I have opted to document this possibility in comments
instead of implementing it in the patch set.
5) + +/* + * GetOldestTransactionIdInCommit() + * + * Similar to GetOldestActiveTransactionId but returns the oldest transaction ID + * that is currently in the commit phase. + */ +TransactionId +GetOldestTransactionIdInCommit(void)If there is no transaction currently in 'commit' phase, this function will then
return the next transaction-id. Please mention this in the comments. I think the
doc 'protocol-replication.html' should also be updated for the same.
I added this info in the doc. But since we have merged this function with
GetOldestActiveTransactionId() which has the same behavior, so I am
not adding more code comments for the existing function.
Best Regards,
Hou zj
On Tue, May 20, 2025 at 11:08 AM shveta malik wrote:
Please find few more comments:
Thanks for the comments!
2)
----------
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 ----------Why do we need this call to 'maybe_advance_nonremovable_xid' towards end
of LogicalRepApplyLoop() i.e. the last call? Can it make any further 'data.phase'
change here? IIUC, there are 2 triggers for 'data.phase' change through
LogicalRepApplyLoop(). First one is for the very first time when we start this
loop, it will set data.phase to
0 (RCI_GET_CANDIDATE_XID) and will call
maybe_advance_nonremovable_xid(). After that, LogicalRepApplyLoop()
function can trigger a 'data.phase' change only when it receives a response
from the publisher. Shouldn't the first 4 calls to
maybe_advance_nonremovable_xid() from LogicalRepApplyLoop() suffice?
I think each invocation of maybe_advance_nonremovable_xid() has a chance to
complete the final RCI_WAIT_FOR_LOCAL_FLUSH phase, as it could be waiting for
changes to be flushed. The function call was added with the intention to
enhance the likelihood of advancing the transaction ID, particularly when it is
waiting for flushed changes. Although we could check the same in other func
calls as well, but I think it's OK to keep the last check.
Best Regards,
Hou zj
On Fri, May 16, 2025 at 7:31 PM Amit Kapila wrote:
A few more comments:
=================
3.
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
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return;/* Exit early if we have already stopped retaining */
if (MyLogicalRepWorker->stop_conflict_info_retention)
return;
...get_candidate_xid()
{
...
if (!TimestampDifferenceExceeds(data->candidate_xid_time, now,
data->xid_advance_interval))
return;Would it be better to encapsulate all these preliminary checks that
decide whether we can move to computing oldest_nonremovable_xid in a
separate function? The check in get_candidate_xid would require some
additional conditions because it is not required in every phase.
Additionally, we can move the core phase processing logic to compute
in a separate function. We can try this to see if the code looks
better with such a refactoring.
I moved the switch case into a separate function process_rci_phase_transition()
and call it in each phase handling function (get_candidate_xid etc).
I also added a new function can_advance_nonremovable_xid to maintain the
preliminary checks. But after re-thinking, the timer check cannot be moved into
this function because it's needed even if being called from the new function
process_rci_phase_transition()->get_candidate_xid() to ensure we do not get
next xid too frequently.
4. + /* + * 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;I think there is a possibility of optimization here for cases where
there are no new transactions on the publisher side across the next
cycle of advancement of oldest_nonremovable_xid. We can simply set
candidate_xid as oldest_nonremovable_xid instead of going into
RCI_WAIT_FOR_LOCAL_FLUSH phase. If you want to keep the code simple
for the first version, then at least note that down in comments, but
OTOH, if it is simple to achieve, then let's do it now.
I think to implement this optimization, it's needed to compare both
remote_nextxid and remote_lsn across consecutive cycles. Although
remote_nextxid might remain unchanged between cycles, old transactions might
have been committed in between two cycles, not affecting nextxid. Therefore,
maintaining two fields last_remote_nextxid and last_remote_lsn within the
structure for comparison is required. Additionally, this optimization implies
skipping the clock skew check in last phase, unless we move the check to a
earlier place. Given that the cost of continuing in RCI_WAIT_FOR_LOCAL_FLUSH
when there's no publisher activity is minimal, I personally prefer keeping the
code simple in this version.
Best Regards,
Hou zj
Thanks you for v31 patch-set. Please find few comments on patch001:
1)
wait_for_local_flush:
+ 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;
+ }
We should only get new flush-position, if 'last_flushpos' is still
lesser than 'data->remote_lsn'. Since 'last_flushpos' is also updated
by 'send_feedback' and we do not update 'data->flushpos_update_time'
there, it is possible that we have latest flush position but still
TimestampDifferenceExceeds gives 'true', making it re-read the flush
position unnecessarily.
Having said that, I think the correct way will be to move
'flushpos_update_time' out of RetainConflictInfoData() similar to
last_flushpos. Let it be a static variable, then we can update it in
send_feedback as well.
2)
get_candidate_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;
+ }
I fail to think of a scenario where oldest_nonremovable_xid can be
greater than current oldest txn-id. I think we should only check for
'equal' in the above condition and assert if oldest_nonremovable_xid
is greater.
3)
+ elog(DEBUG2, "confirmed remote flush up to %X/%X: new
oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(data->remote_lsn),
+ XidFromFullTransactionId(data->candidate_xid));
This message is confusing as we have not confirmed remote flush,
instead it is local flush upton remote-lsn. We can say:
confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
4)
Everywhere we are using variable name as 'data', it is a very generic
name. Shall we change it to 'conflict_info_data' or
'retain_conf_info_data'?
5)
+ * 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 in logical replication. See
+ * GetOldestTransactionIdInCommit and its use.
Please update the comment as 'GetOldestTransactionIdInCommit' is no
longer there.
6)
+ * RecordTransactionCommit uses DELAY_CHKPT_IN_COMMIT for; see notes
+ * there.). Note that DELAY_CHKPT_IN_COMMIT is used to find transactions
there.). --> there.) (another full stop not needed)
thanks
Shveta
On Fri, May 23, 2025 at 12:15 PM shveta malik <shveta.malik@gmail.com> wrote:
Thanks you for v31 patch-set. Please find few comments on patch001:
1)
wait_for_local_flush:
+ 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; + }We should only get new flush-position, if 'last_flushpos' is still
lesser than 'data->remote_lsn'. Since 'last_flushpos' is also updated
by 'send_feedback' and we do not update 'data->flushpos_update_time'
there, it is possible that we have latest flush position but still
TimestampDifferenceExceeds gives 'true', making it re-read the flush
position unnecessarily.Having said that, I think the correct way will be to move
'flushpos_update_time' out of RetainConflictInfoData() similar to
last_flushpos. Let it be a static variable, then we can update it in
send_feedback as well.
But then we may sometimes need to call GetCurrentTimestamp to set its
value, which is not required now. Because we expect to skip this check
only in case when we are frequently applying the changes.
--
With Regards,
Amit Kapila.
On Thu, May 22, 2025 at 8:28 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attaching the V31 patch set which addressed comments in [1]~[8].
Few comments:
1.
<para>
+ The oldest transaction ID along that is currently in the commit
+ phase on the server, along with its epoch.
The first 'along' in the sentence looks redundant. I've removed this
in the attached.
2.
+ data.remote_oldestxid = FullTransactionIdFromU64(pq_getmsgint64(&s));
+ data.remote_nextxid = FullTransactionIdFromU64(pq_getmsgint64(&s));
Shouldn't we need to typecast the result of pq_getmsgint64(&s) with
uint64 as we do at similar other places in pg_snapshot_recv?
3.
+ pq_sendint64(&output_message,
U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, U64FromFullTransactionId(nextFullXid));
Similarly, here also we should typecase with uint64
4.
+ * XXX In phase RCI_REQUEST_PUBLISHER_STATUS, a potential enhancement could be
+ * requesting transaction information specifically for those containing
+ * UPDATEs. However, this approach introduces additional complexities in
+ * tracking UPDATEs for transactions on the publisher, and it may not
+ * effectively address scenarios with frequent UPDATEs.
I think, as the patch needs the oldest_nonremovable_xid idea even to
detect update_origin_differs and delete_origin_differs reliably, as
mentioned in 0001's commit message, is it sufficient to track update
transactions? Don't we need to track it even for deletes? I have
removed this note for now and updated the comment to mention it is
required to detect update_origin_differs and delete_origin_differs
conflicts reliably.
Apart from the above comments, I made a few other cosmetic changes in
the attached.
--
With Regards,
Amit Kapila.
Attachments:
v31_0001_amit.1.patch.txttext/plain; charset=US-ASCII; name=v31_0001_amit.1.patch.txtDownload
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 7a40c62c59b..16d9171bf0d 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2672,11 +2672,11 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<term>Int64</term>
<listitem>
<para>
- The oldest transaction ID along that is currently in the commit
- phase on the server, along with its epoch. The most significant 32
- bits are the epoch. The least significant 32 bits are the
- transaction ID. If no transactions are active on the server, this
- number will be the next transaction ID to be assigned.
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
</para>
</listitem>
</varlistentry>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index e9ca6c88e7e..6b9477bdf6a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,17 +309,16 @@ typedef struct RetainConflictInfoData
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
FullTransactionId candidate_xid; /* candidate for the non-removable
* transaction ID */
+ 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 and the update of the remote flush
- * position.
+ * of transaction ID advancement.
*/
TimestampTz last_recv_time; /* when the last message was received */
TimestampTz candidate_xid_time; /* when the candidate_xid is decided */
- TimestampTz flushpos_update_time; /* when the remote flush position was
- * updated in final phase
- * (RCI_WAIT_FOR_LOCAL_FLUSH) */
int xid_advance_interval; /* how much time (ms) to wait before
* attempting to advance the
* non-removable transaction ID */
@@ -4051,7 +4050,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4106,12 +4108,6 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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.
- *
- * XXX In phase RCI_REQUEST_PUBLISHER_STATUS, a potential enhancement could be
- * requesting transaction information specifically for those containing
- * UPDATEs. However, this approach introduces additional complexities in
- * tracking UPDATEs for transactions on the publisher, and it may not
- * effectively address scenarios with frequent UPDATEs.
*/
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *data,
@@ -4201,8 +4197,7 @@ get_candidate_xid(RetainConflictInfoData *data)
/*
* Immediately update the timer, even if the function returns later without
- * setting candidate_xid due to inactivity on the subscriber. This ensures
- * a certain interval before recalculating candidate_xid, minimizing
+ * setting candidate_xid due to inactivity on the subscriber. This avoids
* frequent calls to GetOldestActiveTransactionId.
*/
data->candidate_xid_time = now;
@@ -4298,8 +4293,8 @@ wait_for_publisher_status(RetainConflictInfoData *data, bool status_received)
* However, it requires maintaining two fields, last_remote_nextxid and
* last_remote_lsn, within the structure for comparison with the current
* cycle's values. Considering the minimal cost of continuing in
- * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opt not to
- * advance transaction ID in this case.
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
*/
if (FullTransactionIdPrecedesOrEquals(data->last_phase_at,
data->remote_oldestxid))
@@ -4399,7 +4394,9 @@ wait_for_local_flush(RetainConflictInfoData *data)
/*
* Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement.
+ * next round of transaction ID advancement. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
*/
data->phase = RCI_GET_CANDIDATE_XID;
data->remote_lsn = InvalidXLogRecPtr;
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index c1d5ddbad4b..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2876,8 +2876,8 @@ GetRunningTransactionData(void)
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
*
- * If inCommitOnly is true, we target transactions currently in the commit
- * phase only.
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
GetOldestActiveTransactionId(bool inCommitOnly)
Hi Zhijie,
Thanks for the effort on the patches. I did a quick look on them before
diving into the logic and discussion. Below are a few minor typos found in
version 31.
⸻
1. Spelling of “non-removable”
[PATCH v31 1/7]
In docs and code “removeable” vs. “removable” are used alternatively and
omitted the hyphen in “non-removable”.
2. Double “arise” in SGML
[PATCH v31 7/7]
In doc/src/sgml/logical-replication.sgml under the <varlistentry
id="conflict-update-deleted">, have duplicate arise:
+ are enabled. Note that if a tuple cannot be found due to the table
being
+ truncated only a <literal>update_missing</literal> conflict will
arise.
+ arise
3. Commit-message typos
[PATCH v31 1/7] (typo “tranasction”)
Subject: [PATCH v30 1/7] Maintain the oldest non removeable tranasction ID
by
apply worker
Show quoted text
Attaching the V31 patch set which addressed comments in [1]~[8].
The comments in [9] concerning the new GUC in patch 0004 is still under
review
and will be addressed in the next version.[1]
/messages/by-id/CAJpy0uD6SgD7w839Wzezdj0JT2OnA+xCxddM15=gb5nRqYAv+A@mail.gmail.com
[2]
/messages/by-id/CAJpy0uCYqG16zCjiCK4og6yqR7zP2rB1oOT7=AnDdVePo-8RrA@mail.gmail.com
[3]
/messages/by-id/CAA4eK1KemsW0EXaSy2Y-M-vVy5Gh4onNG++kKs7ugY+3N-g-Yw@mail.gmail.com
[4]
/messages/by-id/CAA4eK1+r9V6DpH9gYRa2xOx167FapbuKdc4gESr8Etxpx2zrqw@mail.gmail.com
[5]
/messages/by-id/CAJpy0uArh0A9yOxoamD0RWM-7K9kyoUMNh5C2+PFTbGFoxf5wg@mail.gmail.com
[6]
/messages/by-id/CAJpy0uDL4oLdhYup44a2=1OeyUSsKhg8Y30-h1uxcf=mki57uA@mail.gmail.com
[7]
/messages/by-id/CAA4eK1+VNaGi-GU6awgFKmTgidLTHo2HDuzV1+aT8sjn8QtPxg@mail.gmail.com
[8]
/messages/by-id/CAA4eK1+=ZAf0T2iMg2+ZF4cJdUk=UViqpiOg_kPa8jgK+g94aw@mail.gmail.com
[9]
/messages/by-id/CAA4eK1LLaXzsKOaPwGTiikOYySYK+Ty_x3EXg-g=7M_CLn4WiQ@mail.gmail.comBest Regards,
Hou zj
On Fri, May 23, 2025 at 9:21 PM Xuneng Zhou <xunengzhou@gmail.com> wrote:
Looking at v31-0001 again, most of it looks fine except this logic of
getting the commit_ts after marking the transaction in commit. I see
in RecordTransactionCommit(), we are setting this flag
(DELAY_CHKPT_IN_COMMIT) to put the transaction in commit state[1]@@ -1537,7 +1541,7 @@ RecordTransactionCommit(void) */ if (markXidCommitted) { - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION(); }, and
after that we insert the commit log[2]/* * Insert the commit XLOG record. */ XactLogCommitRecord(GetCurrentTransactionStopTimestamp(), nchildren, children, nrels, rels, ndroppedstats, droppedstats, nmsgs, invalMessages, RelcacheInitFileInval, MyXactFlags, InvalidTransactionId, NULL /* plain commit */ );, but I noticed that there we
call GetCurrentTransactionStopTimestamp() for acquiring the commit-ts
and IIUC we want to ensure that commit-ts timestamp should be after we
set the transaction in commit with (DELAY_CHKPT_IN_COMMIT), but
question is, is it guaranteed that the place we are calling
GetCurrentTransactionStopTimestamp() will always give us the current
timestamp? Because if you see this function, it may return
'xactStopTimestamp' as well if that is already set. I am still
digging a bit more. Is there a possibility that 'xactStopTimestamp' is
already set during some interrupt handling when
GetCurrentTransactionStopTimestamp() is already called by
pgstat_report_stat(), or is it guaranteed that during
RecordTransactionCommit we will call this first time?
If we have already ensured this then I think adding a comment to
explain the same will be really useful.
[1]
@@ -1537,7 +1541,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
[2]: /* * Insert the commit XLOG record. */ XactLogCommitRecord(GetCurrentTransactionStopTimestamp(), nchildren, children, nrels, rels, ndroppedstats, droppedstats, nmsgs, invalMessages, RelcacheInitFileInval, MyXactFlags, InvalidTransactionId, NULL /* plain commit */ );
/*
* Insert the commit XLOG record.
*/
XactLogCommitRecord(GetCurrentTransactionStopTimestamp(),
nchildren, children, nrels, rels,
ndroppedstats, droppedstats,
nmsgs, invalMessages,
RelcacheInitFileInval,
MyXactFlags,
InvalidTransactionId, NULL /*
plain commit */ );
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Sat, May 24, 2025 at 10:04 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, May 23, 2025 at 9:21 PM Xuneng Zhou <xunengzhou@gmail.com> wrote:
Looking at v31-0001 again, most of it looks fine except this logic of
getting the commit_ts after marking the transaction in commit. I see
in RecordTransactionCommit(), we are setting this flag
(DELAY_CHKPT_IN_COMMIT) to put the transaction in commit state[1], and
after that we insert the commit log[2], but I noticed that there we
call GetCurrentTransactionStopTimestamp() for acquiring the commit-ts
and IIUC we want to ensure that commit-ts timestamp should be after we
set the transaction in commit with (DELAY_CHKPT_IN_COMMIT), but
question is, is it guaranteed that the place we are calling
GetCurrentTransactionStopTimestamp() will always give us the current
timestamp? Because if you see this function, it may return
'xactStopTimestamp' as well if that is already set. I am still
digging a bit more. Is there a possibility that 'xactStopTimestamp' is
already set during some interrupt handling when
GetCurrentTransactionStopTimestamp() is already called by
pgstat_report_stat(), or is it guaranteed that during
RecordTransactionCommit we will call this first time?If we have already ensured this then I think adding a comment to
explain the same will be really useful.[1] @@ -1537,7 +1541,7 @@ RecordTransactionCommit(void) */ if (markXidCommitted) { - MyProc->delayChkptFlags &= ~DELAY_CHKPT_START; + MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT; END_CRIT_SECTION(); }[2]
/*
* Insert the commit XLOG record.
*/
XactLogCommitRecord(GetCurrentTransactionStopTimestamp(),
nchildren, children, nrels, rels,
ndroppedstats, droppedstats,
nmsgs, invalMessages,
RelcacheInitFileInval,
MyXactFlags,
InvalidTransactionId, NULL /*
plain commit */ );
IMHO, this should not be an issue as the only case where
'xactStopTimestamp' is set for the current process is from
ProcessInterrupts->pgstat_report_stat->
GetCurrentTransactionStopTimestamp, and this call sequence is only
possible when transaction blockState is TBLOCK_DEFAULT. And that is
only set after RecordTransactionCommit() is called, so logically,
RecordTransactionCommit() should always be the first one to set the
'xactStopTimestamp'. But I still think this is a candidate for
comments, or even better,r if somehow it can be ensured by some
assertion, maybe by passing a parameter in
GetCurrentTransactionStopTimestamp() that if this is called from
RecordTransactionCommit() then 'xactStopTimestamp' must not already be
set.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Sat, May 24, 2025 at 10:29 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Sat, May 24, 2025 at 10:04 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, May 23, 2025 at 9:21 PM Xuneng Zhou <xunengzhou@gmail.com> wrote:
Looking at v31-0001 again, most of it looks fine except this logic of
getting the commit_ts after marking the transaction in commit. I see
in RecordTransactionCommit(), we are setting this flag
(DELAY_CHKPT_IN_COMMIT) to put the transaction in commit state[1], and
after that we insert the commit log[2], but I noticed that there we
call GetCurrentTransactionStopTimestamp() for acquiring the commit-ts
and IIUC we want to ensure that commit-ts timestamp should be after we
set the transaction in commit with (DELAY_CHKPT_IN_COMMIT), but
question is, is it guaranteed that the place we are calling
GetCurrentTransactionStopTimestamp() will always give us the current
timestamp? Because if you see this function, it may return
'xactStopTimestamp' as well if that is already set. I am still
digging a bit more. Is there a possibility that 'xactStopTimestamp' is
already set during some interrupt handling when
GetCurrentTransactionStopTimestamp() is already called by
pgstat_report_stat(), or is it guaranteed that during
RecordTransactionCommit we will call this first time?If we have already ensured this then I think adding a comment to
explain the same will be really useful.
...
IMHO, this should not be an issue as the only case where
'xactStopTimestamp' is set for the current process is from
ProcessInterrupts->pgstat_report_stat->
GetCurrentTransactionStopTimestamp, and this call sequence is only
possible when transaction blockState is TBLOCK_DEFAULT. And that is
only set after RecordTransactionCommit() is called, so logically,
RecordTransactionCommit() should always be the first one to set the
'xactStopTimestamp'. But I still think this is a candidate for
comments, or even better,r if somehow it can be ensured by some
assertion, maybe by passing a parameter in
GetCurrentTransactionStopTimestamp() that if this is called from
RecordTransactionCommit() then 'xactStopTimestamp' must not already be
set.
We can add an assertion as you are suggesting, but I feel that adding
a parameter for this purpose looks slightly odd. I suggest adding
comments and probably a test case, if possible, so that if we acquire
commit_ts before setting DELAY_CHKPT_IN_COMMIT flag, then the test
should fail. I haven't checked the feasibility of such a test, so it
may be that it is not feasible or may require some odd injection
points, but even then, it seems better to add comments for this case.
--
With Regards,
Amit Kapila.
On Sat, May 24, 2025 at 11:00 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, May 24, 2025 at 10:29 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Sat, May 24, 2025 at 10:04 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, May 23, 2025 at 9:21 PM Xuneng Zhou <xunengzhou@gmail.com> wrote:
Looking at v31-0001 again, most of it looks fine except this logic of
getting the commit_ts after marking the transaction in commit. I see
in RecordTransactionCommit(), we are setting this flag
(DELAY_CHKPT_IN_COMMIT) to put the transaction in commit state[1], and
after that we insert the commit log[2], but I noticed that there we
call GetCurrentTransactionStopTimestamp() for acquiring the commit-ts
and IIUC we want to ensure that commit-ts timestamp should be after we
set the transaction in commit with (DELAY_CHKPT_IN_COMMIT), but
question is, is it guaranteed that the place we are calling
GetCurrentTransactionStopTimestamp() will always give us the current
timestamp? Because if you see this function, it may return
'xactStopTimestamp' as well if that is already set. I am still
digging a bit more. Is there a possibility that 'xactStopTimestamp' is
already set during some interrupt handling when
GetCurrentTransactionStopTimestamp() is already called by
pgstat_report_stat(), or is it guaranteed that during
RecordTransactionCommit we will call this first time?If we have already ensured this then I think adding a comment to
explain the same will be really useful....
IMHO, this should not be an issue as the only case where
'xactStopTimestamp' is set for the current process is from
ProcessInterrupts->pgstat_report_stat->
GetCurrentTransactionStopTimestamp, and this call sequence is only
possible when transaction blockState is TBLOCK_DEFAULT. And that is
only set after RecordTransactionCommit() is called, so logically,
RecordTransactionCommit() should always be the first one to set the
'xactStopTimestamp'. But I still think this is a candidate for
comments, or even better,r if somehow it can be ensured by some
assertion, maybe by passing a parameter in
GetCurrentTransactionStopTimestamp() that if this is called from
RecordTransactionCommit() then 'xactStopTimestamp' must not already be
set.We can add an assertion as you are suggesting, but I feel that adding
a parameter for this purpose looks slightly odd.
Yeah, that's true. Another option is to add an assert as
Assert(xactStopTimestamp == 0) right before calling
XactLogCommitRecord()? With that, we don't need to pass an extra
parameter, and since we are in a critical section, this process can
not be interrupted, so it's fine even if we have ensured that
'xactStopTimestamp' is 0 before calling the API, as this can not be
changed. And we can add a comment atop this assertion.
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Sat, May 24, 2025 at 3:58 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Sat, May 24, 2025 at 11:00 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, May 24, 2025 at 10:29 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Sat, May 24, 2025 at 10:04 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, May 23, 2025 at 9:21 PM Xuneng Zhou <xunengzhou@gmail.com> wrote:
Looking at v31-0001 again, most of it looks fine except this logic of
getting the commit_ts after marking the transaction in commit. I see
in RecordTransactionCommit(), we are setting this flag
(DELAY_CHKPT_IN_COMMIT) to put the transaction in commit state[1], and
after that we insert the commit log[2], but I noticed that there we
call GetCurrentTransactionStopTimestamp() for acquiring the commit-ts
and IIUC we want to ensure that commit-ts timestamp should be after we
set the transaction in commit with (DELAY_CHKPT_IN_COMMIT), but
question is, is it guaranteed that the place we are calling
GetCurrentTransactionStopTimestamp() will always give us the current
timestamp? Because if you see this function, it may return
'xactStopTimestamp' as well if that is already set. I am still
digging a bit more. Is there a possibility that 'xactStopTimestamp' is
already set during some interrupt handling when
GetCurrentTransactionStopTimestamp() is already called by
pgstat_report_stat(), or is it guaranteed that during
RecordTransactionCommit we will call this first time?If we have already ensured this then I think adding a comment to
explain the same will be really useful....
IMHO, this should not be an issue as the only case where
'xactStopTimestamp' is set for the current process is from
ProcessInterrupts->pgstat_report_stat->
GetCurrentTransactionStopTimestamp, and this call sequence is only
possible when transaction blockState is TBLOCK_DEFAULT. And that is
only set after RecordTransactionCommit() is called, so logically,
RecordTransactionCommit() should always be the first one to set the
'xactStopTimestamp'. But I still think this is a candidate for
comments, or even better,r if somehow it can be ensured by some
assertion, maybe by passing a parameter in
GetCurrentTransactionStopTimestamp() that if this is called from
RecordTransactionCommit() then 'xactStopTimestamp' must not already be
set.We can add an assertion as you are suggesting, but I feel that adding
a parameter for this purpose looks slightly odd.Yeah, that's true. Another option is to add an assert as
Assert(xactStopTimestamp == 0) right before calling
XactLogCommitRecord()? With that, we don't need to pass an extra
parameter, and since we are in a critical section, this process can
not be interrupted, so it's fine even if we have ensured that
'xactStopTimestamp' is 0 before calling the API, as this can not be
changed. And we can add a comment atop this assertion.
This sounds reasonable to me. Let us see what others think.
--
With Regards,
Amit Kapila.
On Sat, May 24, 2025 at 4:46 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
This sounds reasonable to me. Let us see what others think.
I was looking into the for getting the transaction status from
publisher, what I would assume this patch should be doing is request
the publisher status first time, and if some transactions are still in
commit, then we need to wait for them to get completed. But in the
current design its possible that while we are waiting for in-commit
transactions to get committed the old running transaction might come
in commit phase and then we wait for them again, is my understanding
not correct?
Maybe this is very corner case that there are thousands of old running
transaction and everytime we request the status we find some
transactions is in commit phase and the process keep running for long
time until all the old running transaction eventually get committed.
I am thinking can't we make it more deterministic such that when we
get the status first time if we find some transactions that are in
commit phase then we should just wait for those transaction to get
committed? One idea is to get the list of xids in commit phase and
next time when we get the list we can just compare and in next status
if we don't get any xids in commit phase which were in commit phase
during previous status then we are done. But not sure is this worth
the complexity? Mabe not but shall we add some comment explaining the
case and also explaining why this corner case is not harmful?
--
Regards,
Dilip Kumar
EnterpriseDB: http://www.enterprisedb.com
On Fri, May 23, 2025 at 7:08 PM Amit Kapila wrote:
On Thu, May 22, 2025 at 8:28 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attaching the V31 patch set which addressed comments in [1]~[8].
Few comments:
Apart from the above comments, I made a few other cosmetic changes in
the attached.
Thanks for the comments and diff, I have merged them into the latest patch set.
Attaching the V32 patch set which addressed comments in [1]/messages/by-id/CAFiTN-vd=rF0CK1sw-vdOAYSRaV_GMrO3VR0yTv=f2GKyOw=bg@mail.gmail.com~[5]/messages/by-id/CAJpy0uDEKO+qX1Y7WQar_o6nCPBQtenjG=8fE2NvKcwtFRs6hw@mail.gmail.com.
[1]: /messages/by-id/CAFiTN-vd=rF0CK1sw-vdOAYSRaV_GMrO3VR0yTv=f2GKyOw=bg@mail.gmail.com
[2]: /messages/by-id/CAA4eK1Kd05tu9n+a4LAmK-UWxTfcaanuLYD2e1Lt-2kiKzvwTw@mail.gmail.com
[3]: /messages/by-id/CABPTF7U8SKpw+Hepd4MJ+oPBAWdFmYukyk92TKLkZGHhQD4KaQ@mail.gmail.com
[4]: /messages/by-id/CAA4eK1J+dmsB=od4pkGBjZkeurpWGuRXOeXBbNRbh3hVUr7DxQ@mail.gmail.com
[5]: /messages/by-id/CAJpy0uDEKO+qX1Y7WQar_o6nCPBQtenjG=8fE2NvKcwtFRs6hw@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v32-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v32-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 5cbfc5dc7f05b5c3484594c05f63d9e14d7a7c60 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v32 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 2a7a8239a29..3bfe1631934 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 @@ $node_publisher->start;
# 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 @@ $node_subscriber->wait_for_log(
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();
--
2.31.1
v32-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v32-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 705e0d7a8b5e7bf4be2442adb3230c5a0e5805ed Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Mon, 26 May 2025 14:17:01 +0800
Subject: [PATCH v32 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.
---
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 14661ac2cc6..88e2aa43ab0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4944,6 +4944,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal>, as it is
+ reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 574a544d9fa..af4028ce2ba 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29711,7 +29711,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29755,7 +29757,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29785,6 +29789,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for logical replication conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29807,7 +29813,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection. The copied logical slot starts
from the same <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index f0652e30ecf..43f76e8cbae 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2220,6 +2220,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal>, as it
+ is reserved for logical replication conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..eec85cde880 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal>, as it is reserved for
+ logical replication conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 8e42787a426..8cef4460848 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 67655111875..85239f6c316 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 7bf5bfeddfb..2d903c720ec 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4388,6 +4388,9 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
LSN_FORMAT_ARGS(rci_data->remote_lsn),
XidFromFullTransactionId(rci_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. We can even use
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 600b87fa9cb..668279cb4e8 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 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
--
2.31.1
v32-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v32-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From d685ec5c37f3b8c7594e539f0a7081f87f7cce71 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Mon, 26 May 2025 14:18:40 +0800
Subject: [PATCH v32 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 | 35 ++++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 155 ++++++++++++++++--
src/backend/replication/logical/launcher.c | 51 ++++--
src/backend/replication/logical/worker.c | 68 +++++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 4 +
src/test/regress/expected/subscription.out | 179 ++++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
22 files changed, 579 insertions(+), 131 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cbd4e40a320..0fb3894feb6 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8088,6 +8088,18 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index f288c049a5c..9fcb7c0ff73 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
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> is enabled.
</para>
<para>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e5415c3150d 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -285,6 +286,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index eec85cde880..739161df715 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,41 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.<structfield>conflicting</structfield>
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 15efb02badb..637a312e198 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 46d4e65da97..aa353486a4f 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_pub_conflict_info_retention(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,10 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ opts.enabled);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1110,6 +1135,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 +1191,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 +1351,27 @@ 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;
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ sub->enabled);
+
+ /*
+ * 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] =
@@ -1347,6 +1394,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Skip the track_commit_timestamp check by passing it as
+ * true, since it has already been validated during CREATE
+ * SUBSCRIPTION and ALTER SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ true, opts.enabled);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1410,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 +1432,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 +1638,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 +1655,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 +1671,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_pub_conflict_info_retention(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 +2274,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_pub_conflict_info_retention(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 8cef4460848..07d686c6d8c 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 2d903c720ec..6a59db47583 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -445,6 +445,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4111,6 +4112,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4118,9 +4132,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -4549,6 +4567,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 ||
@@ -4558,7 +4585,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,
@@ -5378,6 +5406,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check by passing it as true, since it
+ * has already been validated during CREATE SUBSCRIPTION and ALTER
+ * SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true,
+ false);
+
proc_exit(0);
}
@@ -5732,3 +5768,29 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Give a warning if track_commit_timestamp is not enabled or if the
+ * subscription is disabled.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool track_commit_timestamp,
+ bool sub_enabled)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (!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 (!sub_enabled)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("information for detecting conflicts cannot be purged when the subscription is disabled"));
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index e2e7975b34e..1a34dd2ccb3 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 7417eab6aef..945b3fce670 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 18c2d652bb6..611bc14c1a2 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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 1d08268393e..43a6682a131 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 c916b9299a8..692a7d311cd 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 <name> 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 <name> 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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ae46fa50ce6 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,8 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool track_commit_timestamp,
+ bool sub_enabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..310eb362511 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,45 @@ 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: information for detecting conflicts cannot be purged when the subscription 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 007c9e70374..c65397e5ac6 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;
--
2.31.1
v32-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v32-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From e81b766c3589d05c7284c596148baeaa4ee5114e Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Mon, 26 May 2025 14:27:19 +0800
Subject: [PATCH v32 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 | 183 +++++++++++++++++-
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, 355 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 88e2aa43ab0..ca4625f80bd 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5378,6 +5378,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index d768ea065c5..5a9f17beb23 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
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index b58c52ea50f..fe552a86fad 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 637a312e198..c4443249496 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 07d686c6d8c..f873a817192 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 @@ retry:
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 6a59db47583..3e19b56ae22 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"
@@ -453,8 +454,11 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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,
@@ -3865,7 +3869,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,
@@ -4138,6 +4143,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4292,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->last_phase_at))
rci_data->last_phase_at = rci_data->remote_nextxid;
@@ -4364,6 +4380,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4424,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4447,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4469,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->last_phase_at = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidFullTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4559,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
@@ -4468,6 +4574,51 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_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.
*/
@@ -4639,6 +4790,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();
}
/*
@@ -5285,6 +5446,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 668279cb4e8..bce4af1bdcd 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 2f8cbd86759..e8249ba8491 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 34826d01380..4cabb0e7c48 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 62beb71da28..2e7a337762f 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 c70359d9935..41dc2a2e07c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -121,6 +121,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 6cf828ca8d0..79ed5233edb 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,
--
2.31.1
v32-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v32-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 4d5b46e61068a07f270787580d3f832b3388fa3e Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Mon, 26 May 2025 14:31:20 +0800
Subject: [PATCH v32 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 ca4625f80bd..4c79e1026c1 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5402,7 +5402,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f873a817192..b342cead0a3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -449,7 +449,8 @@ retry:
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->stop_conflict_info_retention = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1287,27 +1288,20 @@ 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)
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
{
- ReplicationSlotRelease();
- InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
- InvalidXLogRecPtr, InvalidOid,
- InvalidTransactionId);
+ if (MyReplicationSlot->data.invalidated == RS_INVAL_NONE)
+ {
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+ }
}
/*
@@ -1316,6 +1310,17 @@ ApplyLauncherMain(Datum main_arg)
*/
else if (nretain_conflict_info)
{
+ /*
+ * Re-create the replication slot if it was invalidated because
+ * all workers stopped conflict retention, and an apply worker has
+ * now resumed the process.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ drop_conflict_slot_if_exists();
+ create_conflict_slot_if_not_exists();
+ }
+
if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 3e19b56ae22..273366a6053 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -458,7 +458,6 @@ static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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,
@@ -4143,10 +4142,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4431,6 +4426,19 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_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 applied and
@@ -4438,6 +4446,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4479,9 +4488,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4510,19 +4518,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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(rci_data);
@@ -4574,51 +4589,6 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_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.
*/
@@ -4790,16 +4760,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();
}
/*
@@ -5446,8 +5406,6 @@ InitializeLogRepWorker(void)
MySubscription->name)));
CommitTransactionCommand();
-
- update_conflict_retention_status();
}
/*
--
2.31.1
v32-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v32-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From be75bcf9ddf0b3a5b750450663b9dd44f34017f4 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 17:33:10 +0800
Subject: [PATCH v32 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 9fcb7c0ff73..a557698b446 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;
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 5a9f17beb23..81b202bbebf 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
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c4443249496..d90289f57da 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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 273366a6053..a8922d895c5 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2813,17 +2813,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. */
@@ -3161,18 +3172,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 97af7c6554f..9f07b2a4b56 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 2e7a337762f..864a923bff1 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 ae99407db89..fcd2d1c2336 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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 3bfe1631934..4eacc0c089a 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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 @@ 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.
+# 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 @@ ok( $stderr =~
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;");
--
2.31.1
v32-0001-Maintain-the-oldest-non-removable-transaction-ID.patchapplication/octet-stream; name=v32-0001-Maintain-the-oldest-non-removable-transaction-ID.patchDownload
From 98920c0c881a0e0c6be6149ab6beaca7e709280c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v32] Maintain the oldest non-removable transaction 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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these 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 | 86 ++++
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 522 ++++++++++++++++++++-
src/backend/replication/walsender.c | 56 +++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/worker_internal.h | 35 ++
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
12 files changed, 750 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 65bbfc34f50..f0652e30ecf 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2638,6 +2638,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2782,6 +2841,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 2d4c346473b..00cab87d106 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 10677da56b2..8e42787a426 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
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 4151a4b2a96..7bf5bfeddfb 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ 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 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,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 +3739,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 +3767,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 +3784,31 @@ 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 = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ 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,409 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+ 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);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = full_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->last_phase_at))
+ rci_data->last_phase_at = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than last_phase_at. This can happen if some old transaction was in the
+ * commit phase when we requested status in this cycle. We do not handle
+ * this case explicitly as it's rare and the benefit doesn't justify the
+ * required complexity. Tracking would require either caching all xids at
+ * the publisher or sending them to subscribers. The condition will
+ * resolve naturally once the remaining transaction finishes.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->last_phase_at,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ FullTransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ XidFromFullTransactionId(rci_data->candidate_xid));
+
+ /*
+ * Reset all data fields except those used to determine the timing for the
+ * next round of transaction ID advancement. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->last_phase_at = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_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 9fa8beb6103..149a8e8a9dd 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..f9aed512c7f 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,41 @@ 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
+ */
+ 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 e5879e00dff..ecf0c2bd701 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
--
2.31.1
On Fri, May 23, 2025 at 2:45 PM shveta malik wrote:
Thanks you for v31 patch-set. Please find few comments on patch001:
Thanks for the comments.
1)
wait_for_local_flush:
+ 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; + }We should only get new flush-position, if 'last_flushpos' is still
lesser than 'data->remote_lsn'. Since 'last_flushpos' is also updated
by 'send_feedback' and we do not update 'data->flushpos_update_time'
there, it is possible that we have latest flush position but still
TimestampDifferenceExceeds gives 'true', making it re-read the flush
position unnecessarily.Having said that, I think the correct way will be to move
'flushpos_update_time' out of RetainConflictInfoData() similar to
last_flushpos. Let it be a static variable, then we can update it in
send_feedback as well.
I added the check to update the flush when last_flushpos is behind.
But I tend to avoid adding more static variables if possible.
The flushpos_update_time is only used to prevent fetching flush position too
frequently which is specific to RCI logic. And even if in some cases, we might
have already updated the flush position in send_feedback(), I feel it's not a
big issue as the new flush position update logic is only used when applying
changes in a loop where send_feedback() is rarely invoked.
4)
Everywhere we are using variable name as 'data', it is a very generic
name. Shall we change it to 'conflict_info_data' or
'retain_conf_info_data'?
I changed them to rci_data which is shorter.
Best Regards,
Hou zj
On Sun, May 25, 2025 at 4:36 PM Dilip Kumar wrote:
On Sat, May 24, 2025 at 4:46 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:This sounds reasonable to me. Let us see what others think.
I was looking into the for getting the transaction status from
publisher, what I would assume this patch should be doing is request
the publisher status first time, and if some transactions are still in
commit, then we need to wait for them to get completed. But in the
current design its possible that while we are waiting for in-commit
transactions to get committed the old running transaction might come
in commit phase and then we wait for them again, is my understanding
not correct?
Thanks for reviewing the patch. And yes, your understanding is correct.
Maybe this is very corner case that there are thousands of old running
transaction and everytime we request the status we find some
transactions is in commit phase and the process keep running for long
time until all the old running transaction eventually get committed.I am thinking can't we make it more deterministic such that when we
get the status first time if we find some transactions that are in
commit phase then we should just wait for those transaction to get
committed? One idea is to get the list of xids in commit phase and
next time when we get the list we can just compare and in next status
if we don't get any xids in commit phase which were in commit phase
during previous status then we are done. But not sure is this worth
the complexity? Mabe not but shall we add some comment explaining the
case and also explaining why this corner case is not harmful?
I also think it's not worth the complexity for this corner case which is
rare. So, I have added some comments in wait_for_publisher_status() to
mention the same.
Best Regards,
Hou zj
On Sat, May 24, 2025 at 6:28 PM Dilip Kumar wrote:
On Sat, May 24, 2025 at 11:00 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Sat, May 24, 2025 at 10:29 AM Dilip Kumar <dilipbalaut@gmail.com>
wrote:
On Sat, May 24, 2025 at 10:04 AM Dilip Kumar <dilipbalaut@gmail.com>
wrote:
On Fri, May 23, 2025 at 9:21 PM Xuneng Zhou
<xunengzhou@gmail.com> wrote:
Looking at v31-0001 again, most of it looks fine except this logic
of getting the commit_ts after marking the transaction in commit.
I see in RecordTransactionCommit(), we are setting this flag
(DELAY_CHKPT_IN_COMMIT) to put the transaction in commit state[1],
and after that we insert the commit log[2], but I noticed that
there we call GetCurrentTransactionStopTimestamp() for acquiring
the commit-ts and IIUC we want to ensure that commit-ts timestamp
should be after we set the transaction in commit with
(DELAY_CHKPT_IN_COMMIT), but question is, is it guaranteed that
the place we are calling
GetCurrentTransactionStopTimestamp() will always give us the
current timestamp? Because if you see this function, it may return
'xactStopTimestamp' as well if that is already set. I am still
digging a bit more. Is there a possibility that
'xactStopTimestamp' is already set during some interrupt handling
when
GetCurrentTransactionStopTimestamp() is already called by
pgstat_report_stat(), or is it guaranteed that during
RecordTransactionCommit we will call this first time?If we have already ensured this then I think adding a comment to
explain the same will be really useful....
IMHO, this should not be an issue as the only case where
'xactStopTimestamp' is set for the current process is from
ProcessInterrupts->pgstat_report_stat->
GetCurrentTransactionStopTimestamp, and this call sequence is only
possible when transaction blockState is TBLOCK_DEFAULT. And that is
only set after RecordTransactionCommit() is called, so logically,
RecordTransactionCommit() should always be the first one to set the
'xactStopTimestamp'. But I still think this is a candidate for
comments, or even better,r if somehow it can be ensured by some
assertion, maybe by passing a parameter in
GetCurrentTransactionStopTimestamp() that if this is called from
RecordTransactionCommit() then 'xactStopTimestamp' must not already
be set.We can add an assertion as you are suggesting, but I feel that adding
a parameter for this purpose looks slightly odd.Yeah, that's true. Another option is to add an assert as
Assert(xactStopTimestamp == 0) right before calling
XactLogCommitRecord()? With that, we don't need to pass an extra
parameter, and since we are in a critical section, this process can not be
interrupted, so it's fine even if we have ensured that 'xactStopTimestamp' is 0
before calling the API, as this can not be changed. And we can add a
comment atop this assertion.
Thanks for the suggestion !
I think adding an Assert as suggested is OK. I am not adding more comments
atop of the assert because we already have comments in a very close place that
explains the importance of setting the flag first.
Best Regards,
Hou zj
On Fri, May 23, 2025 at 11:51 PM Xuneng Zhou wrote:
Thanks for the effort on the patches. I did a quick look on them before
diving into the logic and discussion. Below are a few minor typos found in
version 31.
Thanks for the comments! I have fixed these typos in latest version.
Best Regards,
Hou zj
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Sun, May 25, 2025 at 4:36 PM Dilip Kumar wrote:
I am thinking can't we make it more deterministic such that when we
get the status first time if we find some transactions that are in
commit phase then we should just wait for those transaction to get
committed? One idea is to get the list of xids in commit phase and
next time when we get the list we can just compare and in next status
if we don't get any xids in commit phase which were in commit phase
during previous status then we are done. But not sure is this worth
the complexity? Mabe not but shall we add some comment explaining the
case and also explaining why this corner case is not harmful?I also think it's not worth the complexity for this corner case which is
rare.
Yeah, complexity is one part, but I feel improving such less often
cases could add performance burden for more often cases where we need
to either maintain and invalidate the cache on the publisher or send
the list of all such xids to the subscriber over the network.
So, I have added some comments in wait_for_publisher_status() to
mention the same.
I agree that at this stage it is good to note down in comments, and if
we face such cases often, then we can improve it in the future.
--
With Regards,
Amit Kapila.
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attaching the V32 patch set which addressed comments in [1]~[5].
Thanks for the patch, I am still reviewing the patches, please find
few trivial comments for patch001:
1)
+ FullTransactionId last_phase_at; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
'last_phase_at' seems like we are talking about the phase in the past.
(similar to 'last' in last_recv_time).
Perhaps we should name it as 'final_phase_at'
2)
RetainConflictInfoData data = {0};
We can change this name as well to rci_data.
3)
+ /*
+ * 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);
Shall we name it to full_oldest_xid for better clarity?
4)
+ /*
+ * 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 (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_data->last_recv_time, WalWriterDelay))
a) costy --> costly
thanks
Shveta
On Tue, May 27, 2025 at 3:59 PM shveta malik <shveta.malik@gmail.com> wrote:
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attaching the V32 patch set which addressed comments in [1]~[5].
Thanks for the patch, I am still reviewing the patches, please find
few trivial comments for patch001:1)
+ FullTransactionId last_phase_at; /* publisher transaction ID that must + * be awaited to complete before + * entering the final phase + * (RCI_WAIT_FOR_LOCAL_FLUSH) */'last_phase_at' seems like we are talking about the phase in the past.
(similar to 'last' in last_recv_time).
Perhaps we should name it as 'final_phase_at'
I am not sure the phase in this variable name matches with what it is
used for. The other option could be remote_wait_for or something on
those lines.
Additionally, please find a few cosmetic changes atop 0001 and 0002
patches. Please include in next set, if those looks okay to you.
--
With Regards,
Amit Kapila.
Attachments:
v32_amit_1.patch.txttext/plain; charset=US-ASCII; name=v32_amit_1.patch.txtDownload
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 740ec89e070..fe558f0a81c 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,8 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
- The name cannot be <literal>pg_conflict_detection</literal>, as it is
- reserved for logical replication conflict detection.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 4d98bf9bf19..5866dff9490 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29711,8 +29711,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new physical replication slot named
<parameter>slot_name</parameter>. The name cannot be
- <literal>pg_conflict_detection</literal>, as it is reserved for
- logical replication conflict detection. The optional second parameter,
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29757,8 +29757,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
<parameter>plugin</parameter>. The name cannot be
- <literal>pg_conflict_detection</literal>, as it is reserved for
- logical replication conflict detection. The optional third
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29789,7 +29789,7 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
The new slot name cannot be <literal>pg_conflict_detection</literal>,
- as it is reserved for logical replication conflict detection.
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29813,9 +29813,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
the output plugin and persistence. The name cannot be
- <literal>pg_conflict_detection</literal>, as it is reserved for
- logical replication conflict detection. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 16d9171bf0d..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,8 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
- The name cannot be <literal>pg_conflict_detection</literal>, as it
- is reserved for logical replication conflict detection.
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 739161df715..dba0f541ac5 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -170,8 +170,8 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<para>
Name of the publisher's replication slot to use. The default is
to use the name of the subscription for the slot name. The name cannot
- be <literal>pg_conflict_detection</literal>, as it is reserved for
- logical replication conflict detection.
+ be <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6a59db47583..17963486795 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3862,8 +3862,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
wait_time = NAPTIME_PER_CYCLE;
/*
- * Ensure to wake up when it's possible to attempt to advance the
- * non-removable transaction ID.
+ * Ensure to wake up when it's possible 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);
@@ -4103,11 +4103,13 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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.
+ * XXX It seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
*/
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
@@ -4211,6 +4213,10 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
*/
full_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+ /*
+ * Oldest active transaction ID (full_xid) can't be behind any of its
+ * previously computed value.
+ */
Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
full_xid));
@@ -4294,12 +4300,12 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
*
* It's possible that transactions in the commit phase during the last
* cycle have now finished committing, but remote_oldestxid remains older
- * than last_phase_at. This can happen if some old transaction was in the
+ * than last_phase_at. This can happen if some old transaction came in the
* commit phase when we requested status in this cycle. We do not handle
* this case explicitly as it's rare and the benefit doesn't justify the
* required complexity. Tracking would require either caching all xids at
* the publisher or sending them to subscribers. The condition will
- * resolve naturally once the remaining transaction finishes.
+ * resolve naturally once the remaining transactions are finished.
*
* Directly advancing the non-removable transaction ID is possible if
* there are no activities on the publisher since the last advancement
On Tue, May 27, 2025 at 11:45 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Sun, May 25, 2025 at 4:36 PM Dilip Kumar wrote:
I am thinking can't we make it more deterministic such that when we
get the status first time if we find some transactions that are in
commit phase then we should just wait for those transaction to get
committed? One idea is to get the list of xids in commit phase and
next time when we get the list we can just compare and in next status
if we don't get any xids in commit phase which were in commit phase
during previous status then we are done. But not sure is this worth
the complexity? Mabe not but shall we add some comment explaining the
case and also explaining why this corner case is not harmful?I also think it's not worth the complexity for this corner case which is
rare.Yeah, complexity is one part, but I feel improving such less often
cases could add performance burden for more often cases where we need
to either maintain and invalidate the cache on the publisher or send
the list of all such xids to the subscriber over the network.
Yeah, that's a valid point.
So, I have added some comments in wait_for_publisher_status() to
mention the same.I agree that at this stage it is good to note down in comments, and if
we face such cases often, then we can improve it in the future.
+1
--
Regards,
Dilip Kumar
Google
On Tue, May 27, 2025 at 3:59 PM shveta malik <shveta.malik@gmail.com> wrote:
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attaching the V32 patch set which addressed comments in [1]~[5].
Thanks for the patch, I am still reviewing the patches, please find
few trivial comments for patch001:1)
+ FullTransactionId last_phase_at; /* publisher transaction ID that must + * be awaited to complete before + * entering the final phase + * (RCI_WAIT_FOR_LOCAL_FLUSH) */'last_phase_at' seems like we are talking about the phase in the past.
(similar to 'last' in last_recv_time).
Perhaps we should name it as 'final_phase_at'2)
RetainConflictInfoData data = {0};We can change this name as well to rci_data.
3) + /* + * 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);Shall we name it to full_oldest_xid for better clarity?
4) + /* + * 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 (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time && + TimestampDifferenceExceeds(rci_data->flushpos_update_time, + rci_data->last_recv_time, WalWriterDelay))a) costy --> costly
Please find few comments in docs of v32-003:
1)
logical-replication.sgml:
<para>
<link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
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
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
is enabled.
</para>
Above doc is updated under Publishers section:
<sect2 id="logical-replication-config-publisher">
<title>Publishers</title>
But the conflict-slot is created on subscriber, so this info shall be
moved to subscriber. But subscriber currently does not have
'max_replication_slots' parameter under it. But I guess with
conflict-slot created on subscribers, we need to have that parameter
under 'Subscribers' too.
2)
catalogs.sgml:
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
+ </para></entry>
+ </row>
+
2a) In absence of rest of the patches atop 3rd patch, this failed to
compile due to missing xref link. Error:
element xref: validity error : IDREF attribute linked references an unknown ID "conflict-update-deleted"
2b) Also, if 'subretainconflictinfo' is true, IMO, it does not enable
the update_deleted detection, it just provides information which can
be used for detection. We should rephrase this doc.
3)
<xref linkend="conflict-update-deleted"/> is used in
create_subscription.sgml as well. That too needs correction.
4)
+ if (!sub_enabled)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("information for detecting conflicts cannot be purged when
the subscription is disabled"));
WARNING is good but not enough to clarify the problem and our
recommendation in such a case. Shall we update the docs as well as
explaining that such a situation may result in system bloat and thus
if subscription is disabled for longer, it is good to have
retain_conflict_info disabled as well.
thanks
Shveta
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attaching the V32 patch set which addressed comments in [1]~[5].
Review comments:
===============
*
+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;
The above code suggests that the launcher could compute a new xmin
that is less than slot's xmin. At first, this looks odd to me, but
IIUC, this can happen when the user toggles retain_conflict_info flag
at some random time when the launcher is trying to compute the new
xmin value for the slot. One of the possible combinations of steps for
this race could be as follows:
1. The subscriber has two subscriptions, A and B. Subscription A has
retain_conflict_info as true, and B has retain_conflict_info as false
2. Say the launcher calls get_subscription_list(), and worker A is
already alive.
3. Assuming the apply worker will restart on changing
retain_conflict_info, the user enables retain_conflict_info for
subscription B.
4. The launcher processes the subscription B first in the first cycle,
and starts worker B. Say, worker B gets 759 as candidate_xid.
5. The launcher creates the conflict detection slot, xmin = 759
6. Say a new txn happens, worker A gets 760 as candidate_xid and
updates it to oldest_nonremovable_xid.
7. The launcher processes the subscription A in the first cycle, and
the final xmin value is 760, because it only checks the
oldest_nonremovable_xid from worker A. The launcher then updates the
value to slot.xmin.
8. In the next cycle, the launcher finds that worker B has an older
oldest_nonremovable_xid 759, so the minimal xid would now be 759. The
launher would have retreated the slot's xmin unless we had the above
check in the quoted code.
I think the above race is possible because the system lets the changed
subscription values of a subscription take effect asynchronously by
workers. The one more similar race condition handled by the patch is
as follows:
*
...
+ * 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
...
+ FullTransactionId oldest_nonremovable_xid;
This case can happen if the user disables and immediately enables the
retain_conflict_info option. In this case, the launcher may drop the
slot after noticing the disable. But the apply worker may not notice
the disable and it only notices that the retain_conflict_info is still
enabled, so it will keep maintaining oldest_nonremovable_xid when the
slot is not created.
It is okay to handle both the race conditions, but I am worried we may
miss some such race conditions which could lead to difficult-to-find
bugs. So, at least for the first version of this option (aka for
patches 0001 to 0003), we can add a condition that allows us to change
retain_conflict_info only on disabled subscriptions. This will
simplify the patch. We can make a separate patch to allow changing
retain_conflict_info option for enabled subscriptions. That will make
it easier to evaluate such race conditions and the solutions more
deeply.
--
With Regards,
Amit Kapila.
On Mon, Jun 2, 2025 at 12:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attaching the V32 patch set which addressed comments in [1]~[5].
Review comments: =============== * +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;The above code suggests that the launcher could compute a new xmin
that is less than slot's xmin. At first, this looks odd to me, but
IIUC, this can happen when the user toggles retain_conflict_info flag
at some random time when the launcher is trying to compute the new
xmin value for the slot. One of the possible combinations of steps for
this race could be as follows:1. The subscriber has two subscriptions, A and B. Subscription A has
retain_conflict_info as true, and B has retain_conflict_info as false
2. Say the launcher calls get_subscription_list(), and worker A is
already alive.
3. Assuming the apply worker will restart on changing
retain_conflict_info, the user enables retain_conflict_info for
subscription B.
4. The launcher processes the subscription B first in the first cycle,
and starts worker B. Say, worker B gets 759 as candidate_xid.
5. The launcher creates the conflict detection slot, xmin = 759
6. Say a new txn happens, worker A gets 760 as candidate_xid and
updates it to oldest_nonremovable_xid.
7. The launcher processes the subscription A in the first cycle, and
the final xmin value is 760, because it only checks the
oldest_nonremovable_xid from worker A. The launcher then updates the
value to slot.xmin.
8. In the next cycle, the launcher finds that worker B has an older
oldest_nonremovable_xid 759, so the minimal xid would now be 759. The
launher would have retreated the slot's xmin unless we had the above
check in the quoted code.I think the above race is possible because the system lets the changed
subscription values of a subscription take effect asynchronously by
workers. The one more similar race condition handled by the patch is
as follows:* ... + * 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 + * w.r.t the transaction ID stored in the slot maintained by launcher. See + * advance_conflict_slot_xmin. ... + FullTransactionId oldest_nonremovable_xid;This case can happen if the user disables and immediately enables the
retain_conflict_info option. In this case, the launcher may drop the
slot after noticing the disable. But the apply worker may not notice
the disable and it only notices that the retain_conflict_info is still
enabled, so it will keep maintaining oldest_nonremovable_xid when the
slot is not created.
Another case to handle is similar to above with only difference that
no transaction ID wraparound has happened. In such a case, the
launcher may end-up using worker's oldest_nonremovable_xid from the
previous cycle before user disabled-enabled retain_conflict_info. This
may result in the slot moving backward in absence of suggested check
in advance_conflict_slot_xmin(),
thanks
Shveta
On Mon, Jun 2, 2025 at 2:39 PM Amit Kapila wrote:
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attaching the V32 patch set which addressed comments in [1]~[5].
Review comments: =============== * +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;The above code suggests that the launcher could compute a new xmin that is
less than slot's xmin. At first, this looks odd to me, but IIUC, this can happen
when the user toggles retain_conflict_info flag at some random time when the
launcher is trying to compute the new xmin value for the slot. One of the
possible combinations of steps for this race could be as follows:1. The subscriber has two subscriptions, A and B. Subscription A has
retain_conflict_info as true, and B has retain_conflict_info as false 2. Say the
launcher calls get_subscription_list(), and worker A is already alive.
3. Assuming the apply worker will restart on changing retain_conflict_info, the
user enables retain_conflict_info for subscription B.
4. The launcher processes the subscription B first in the first cycle, and starts
worker B. Say, worker B gets 759 as candidate_xid.
5. The launcher creates the conflict detection slot, xmin = 759 6. Say a new txn
happens, worker A gets 760 as candidate_xid and updates it to
oldest_nonremovable_xid.
7. The launcher processes the subscription A in the first cycle, and the final
xmin value is 760, because it only checks the oldest_nonremovable_xid from
worker A. The launcher then updates the value to slot.xmin.
8. In the next cycle, the launcher finds that worker B has an older
oldest_nonremovable_xid 759, so the minimal xid would now be 759. The
launher would have retreated the slot's xmin unless we had the above check in
the quoted code.I think the above race is possible because the system lets the changed
subscription values of a subscription take effect asynchronously by workers.
The one more similar race condition handled by the patch is as follows:* ... + * 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 + * w.r.t the transaction ID stored in the slot maintained by launcher. + See + * advance_conflict_slot_xmin. ... + FullTransactionId oldest_nonremovable_xid;This case can happen if the user disables and immediately enables the
retain_conflict_info option. In this case, the launcher may drop the slot after
noticing the disable. But the apply worker may not notice the disable and it only
notices that the retain_conflict_info is still enabled, so it will keep maintaining
oldest_nonremovable_xid when the slot is not created.It is okay to handle both the race conditions, but I am worried we may miss
some such race conditions which could lead to difficult-to-find bugs. So, at
least for the first version of this option (aka for patches 0001 to 0003), we can
add a condition that allows us to change retain_conflict_info only on disabled
subscriptions. This will simplify the patch.
Agreed.
We can make a separate patch to
allow changing retain_conflict_info option for enabled subscriptions. That will
make it easier to evaluate such race conditions and the solutions more deeply.
I will prepare a separate patch soon.
Here is the V33 patch set which includes the following changes:
0001:
* Renaming and typo fixes based on Shveta's comments [1]/messages/by-id/CAJpy0uBSsRuVOeuo-i8R_aO0CMiORHTnEBZ9z-TDq941WqhyLA@mail.gmail.com
* Comment changes suggested by Amit [2]/messages/by-id/CAA4eK1KUTHbgroBRNp8_dy3Lrc+etPm19O1RcyRcDBgCp7EFcg@mail.gmail.com
* Changed oldest_nonremoable_xid from FullTransactionID to TransactionID.
* Code refactoring in drop_conflict_slot_if_exists()
0002:
* Documentation updates suggested by Amit [2]/messages/by-id/CAA4eK1KUTHbgroBRNp8_dy3Lrc+etPm19O1RcyRcDBgCp7EFcg@mail.gmail.com
* Code modifications to adapt to TransactionID oldest_nonremoable_xid
0003:
* Documentation improvements suggested by Shveta [3]/messages/by-id/CAJpy0uAJUTmSx7fAE3gbnBUzp9ZDOgkLrP5gdoysKUGbvf7vGg@mail.gmail.com
* Added restriction: disallow changing retain_conflict_info when sub
is enabled or worker is alive
0004:
* Simplified race condition handling due to the new restriction from 0003
0005:
* Code updates to accommodate both the TransactionID type for
oldest_nonremoable_xid and the new restriction from 0003
0006:
* New test case for the restriction introduced in 0003
0007:
No changes
[1]: /messages/by-id/CAJpy0uBSsRuVOeuo-i8R_aO0CMiORHTnEBZ9z-TDq941WqhyLA@mail.gmail.com
[2]: /messages/by-id/CAA4eK1KUTHbgroBRNp8_dy3Lrc+etPm19O1RcyRcDBgCp7EFcg@mail.gmail.com
[3]: /messages/by-id/CAJpy0uAJUTmSx7fAE3gbnBUzp9ZDOgkLrP5gdoysKUGbvf7vGg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v33-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v33-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From cd49cdaad0ee2b55a9eaa0968642ddf1593ad270 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 4 Jun 2025 16:40:43 +0800
Subject: [PATCH v33 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 10 +-
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 +++-
14 files changed, 268 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 99f79caf3f7..0fb3894feb6 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8093,7 +8093,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 5073d31ca8e..f75ae3f01f2 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index be90088bcd0..778b51e335c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,10 +445,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
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
- <literal>false</literal>. If set to true, an additional replication
- slot named <quote><literal>pg_conflict_detection</literal></quote>
- will be created on the subscriber to prevent the conflict information
- from being removed.
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
</para>
<para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 a85858df977..6f4b589d233 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2815,17 +2815,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. */
@@ -3163,18 +3174,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 1c12ddbae49..2a084d3f3f0 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 846023894cc..6b375f555a8 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 7706465b351..65c98c44cac 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -239,6 +242,8 @@ ok( $stderr =~
###############################################################################
# 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
@@ -267,10 +272,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v33-0001-Maintain-the-oldest-non-removable-transaction-ID.patchapplication/octet-stream; name=v33-0001-Maintain-the-oldest-non-removable-transaction-ID.patchDownload
From d4faaecca0e2790b9c0279f292865c780b16099c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v33 1/7] Maintain the oldest non-removable transaction 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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these 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 | 86 ++++
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 520 ++++++++++++++++++++-
src/backend/replication/walsender.c | 56 +++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/worker_internal.h | 11 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
12 files changed, 724 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..ed56df85ca7 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2643,6 +2643,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2846,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 2e67e998adb..fa3f5a4315c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1459,10 +1459,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1565,7 +1571,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1914859b2ee..5f19981f9c3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 10677da56b2..9b155705dbb 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
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 a23262957ac..a4f4772fdd5 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,406 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any of
+ * its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in the
+ * commit phase when we requested status in this cycle. We do not handle
+ * this case explicitly as it's rare and the benefit doesn't justify the
+ * required complexity. Tracking would require either caching all xids at
+ * the publisher or sending them to subscribers. The condition will
+ * resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_data->candidate_xid);
+
+ /*
+ * Reset all data fields except those used to determine the timing for the
+ * next round of transaction ID advancement. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_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 f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..c9ef5259b68 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,17 @@ 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 a8346cda633..b77253e35e4 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2554,6 +2554,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v33-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v33-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 2e226b6bfd8bf795cf472be03936f3c990dbe81c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 3 Jun 2025 11:40:30 +0800
Subject: [PATCH v33 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.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
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 | 170 +++++++++++++++++-
.../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, 236 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 021153b2a5f..20b74d92006 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,6 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c67688cbf5f..1fd0f908554 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index ed56df85ca7..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 9b155705dbb..4dcc6031659 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"
@@ -90,8 +91,12 @@ static dshash_table *last_start_times = NULL;
static bool on_commit_launcher_wakeup = false;
+/*
+ * Whether the slot used to retain dead tuples for conflict detection has been
+ * dropped.
+ */
+static bool conflict_slot_dropped = 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 +105,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(TransactionId new_xmin);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1114,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1147,6 +1158,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1166,15 +1179,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)
+ {
+ TransactionId 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 (!TransactionIdIsValid(nonremovable_xid))
+ can_advance_xmin = false;
+ else if (!TransactionIdIsValid(xmin) ||
+ TransactionIdPrecedes(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 +1261,16 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (sublist && can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ else if (!sublist)
+ drop_conflict_slot_if_exists();
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1234,6 +1298,108 @@ 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();
+
+ conflict_slot_dropped = false;
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
+ new_xmin));
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->data.xmin = 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)
+{
+ /*
+ * Avoid the overhead of scanning shared memory for a replication slot that
+ * is known to have been dropped.
+ */
+ if (conflict_slot_dropped)
+ return;
+
+ if (MyReplicationSlot)
+ ReplicationSlotDropAcquired();
+ else if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ ReplicationSlotDrop(CONFLICT_DETECTION_SLOT, true);
+
+ conflict_slot_dropped = 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 67655111875..85239f6c316 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 a4f4772fdd5..e56b4e06767 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4386,6 +4386,9 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
LSN_FORMAT_ARGS(rci_data->remote_lsn),
rci_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. We can even use
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 600b87fa9cb..668279cb4e8 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 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
--
2.30.0.windows.2
v33-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v33-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From ad6d309781f60772c7d5a197ba178902d4184bea Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 4 Jun 2025 16:08:45 +0800
Subject: [PATCH v33 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 | 11 ++
doc/src/sgml/logical-replication.sgml | 6 +
doc/src/sgml/ref/alter_subscription.sgml | 17 +-
doc/src/sgml/ref/create_subscription.sgml | 35 ++++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 202 +++++++++++++++++++--
src/backend/replication/logical/launcher.c | 58 ++++--
src/backend/replication/logical/worker.c | 75 +++++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 4 +
src/test/regress/expected/subscription.out | 179 ++++++++++--------
src/test/regress/sql/subscription.sql | 16 ++
22 files changed, 644 insertions(+), 134 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index cbd4e40a320..99f79caf3f7 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8088,6 +8088,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is still useful for conflict detection
+ is retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 686dd441d02..5073d31ca8e 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..9d4731441ff 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,13 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..be90088bcd0 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,41 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, an additional replication
+ slot named <quote><literal>pg_conflict_detection</literal></quote>
+ will be created on the subscriber to prevent the conflict information
+ from being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..dd3939fba03 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_pub_conflict_info_retention(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,10 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ opts.enabled);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1110,6 +1135,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 +1191,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 +1351,74 @@ 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;
+
+ /*
+ * Do not allow changing the option when the subscription
+ * is enabled and the apply worker is active, to prevent
+ * race conditions arising from the new option value being
+ * acknowledged asynchronously by the launcher and apply
+ * workers.
+ *
+ * Without the restriction, a race condition may arise when
+ * a user disables and immediately re-enables the
+ * retain_conflict_info option. In this case, the launcher
+ * might drop the slot upon noticing the disable action,
+ * while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option
+ * change. During this period, a transaction ID wraparound
+ * could falsely make this ID appear as if it originates
+ * from the future w.r.t the transaction ID stored in the
+ * slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Similarily, if the user enables retain_conflict_info
+ * concurrently with the launcher starting the worker, the
+ * apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update
+ * slot.xmin to a newer value than that maintained by the
+ * worker. In subsequent cycles, upon integrating the
+ * worker's oldest_nonremovable_xid, the launcher might
+ * detect a regression in the calculated xmin,
+ * necessitating additional handling.
+ *
+ * While defining oldest_nonremovable_xid using
+ * FullTransactionID and checks against backward shifts in
+ * slot.xmin could address these concerns, we prioritize
+ * simplicity.
+ */
+ if (sub->enabled)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot set option %s for enabled subscription",
+ "retain_conflict_info")));
+
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ sub->enabled);
+
+ /*
+ * 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] =
@@ -1347,6 +1441,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Skip the track_commit_timestamp check by passing it as
+ * true, since it has already been validated during CREATE
+ * SUBSCRIPTION and ALTER SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ true, opts.enabled);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1457,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 +1479,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 +1685,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 +1702,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 +1718,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_pub_conflict_info_retention(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 +2321,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_pub_conflict_info_retention(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 4dcc6031659..80bbcd5eb9a 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -92,8 +92,8 @@ static dshash_table *last_start_times = NULL;
static bool on_commit_launcher_wakeup = false;
/*
- * Whether the slot used to retain dead tuples for conflict detection has been
- * dropped.
+ * Whether the slot used to retain information useful for conflict detection
+ * has been dropped.
*/
static bool conflict_slot_dropped = false;
@@ -156,6 +156,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);
@@ -449,7 +450,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = MyReplicationSlot
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1159,6 +1162,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1180,16 +1184,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);
@@ -1199,10 +1213,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)
{
TransactionId nonremovable_xid;
@@ -1228,7 +1243,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,
@@ -1266,9 +1282,9 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed. Otherwise, drop the slot if we're no longer
* retaining information useful for conflict detection.
*/
- if (sublist && can_advance_xmin)
+ if (retain_conflict_info && can_advance_xmin)
advance_conflict_slot_xmin(xmin);
- else if (!sublist)
+ else if (!retain_conflict_info)
drop_conflict_slot_if_exists();
/* Switch back to original memory context. */
@@ -1299,7 +1315,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
@@ -1343,16 +1359,18 @@ 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(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
new_xmin));
+ if (!TransactionIdIsValid(new_xmin))
+ return;
+
SpinLockAcquire(&MyReplicationSlot->mutex);
MyReplicationSlot->data.xmin = new_xmin;
SpinLockRelease(&MyReplicationSlot->mutex);
@@ -1379,8 +1397,8 @@ advance_conflict_slot_xmin(TransactionId 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 e56b4e06767..99b101f8766 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"
@@ -445,6 +446,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4116,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4136,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5255,24 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info is enabled at startup without
+ * slot creation to prevent unnecessary maintenance of
+ * oldest_nonremovable_xid. The oldest_nonremovable_xid should be
+ * initialized only when the slot exists. See logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart",
+ MySubscription->name),
+ errdetail("The replication slot \"%s\" has not been created, yet the option %s was enabled during startup.",
+ CONFLICT_DETECTION_SLOT, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5429,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check by passing it as true, since it
+ * has already been validated during CREATE SUBSCRIPTION and ALTER
+ * SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true,
+ false);
+
proc_exit(0);
}
@@ -5754,3 +5799,29 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Give a warning if track_commit_timestamp is not enabled or if the
+ * subscription is disabled.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool track_commit_timestamp,
+ bool sub_enabled)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (!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 (!sub_enabled)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("information for detecting conflicts cannot be purged when the subscription is disabled"));
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 37432e66efd..13d57dd3b13 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 7417eab6aef..945b3fce670 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 940fc77fc2e..480dc126a5f 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();
}
@@ -2017,9 +2017,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)
@@ -2027,6 +2028,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)
@@ -2052,6 +2055,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);
@@ -2114,6 +2142,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 1d08268393e..43a6682a131 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 ec65ab79fec..cbf2703c190 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3728,8 +3729,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ae46fa50ce6 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,8 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool track_commit_timestamp,
+ bool sub_enabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..310eb362511 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,45 @@ 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: information for detecting conflicts cannot be purged when the subscription 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v33-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v33-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From b4d955bdecbc0edb94bb45f03e5df64c76caeaa3 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 4 Jun 2025 16:31:50 +0800
Subject: [PATCH v33 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 | 98 ++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 311 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20b74d92006..8712894a791 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5395,6 +5395,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration (in milliseconds) for which conflict
+ information can be retained for conflict detection by the apply worker.
+ The default value is <literal>0</literal>, indicating that conflict
+ information is retained until it is no longer needed for detection
+ purposes.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index b58c52ea50f..fe552a86fad 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 80bbcd5eb9a..f5f3c87042b 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;
@@ -108,7 +111,7 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
-
+static void invalidate_conflict_slot(void);
/*
* Load the list of subscriptions.
@@ -318,6 +321,8 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
TimestampTz now;
bool is_tablesync_worker = (wtype == WORKERTYPE_TABLESYNC);
bool is_parallel_apply_worker = (wtype == WORKERTYPE_PARALLEL_APPLY);
+ bool retaining_conflict_info = (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
/*----------
* Sanity checks:
@@ -450,9 +455,10 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = MyReplicationSlot
+ worker->oldest_nonremovable_xid = retaining_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = retaining_conflict_info;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1162,7 +1168,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;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1191,7 +1198,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1217,22 +1224,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)
{
TransactionId 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 (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
can_advance_xmin = false;
- else if (!TransactionIdIsValid(xmin) ||
- TransactionIdPrecedes(nonremovable_xid, xmin))
+ else if (can_advance_xmin &&
+ (!TransactionIdIsValid(xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, xmin)))
xmin = nonremovable_xid;
}
@@ -1277,14 +1294,22 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Invalidate the conflict slot if all workers with
+ * retain_conflict_info enabled have stopped further conflict
+ * retention.
+ */
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
+ invalidate_conflict_slot();
+
/*
* Maintain the xmin value of the replication slot for conflict
* detection if needed. Otherwise, drop the slot if we're no longer
* retaining information useful for conflict detection.
*/
- if (retain_conflict_info && can_advance_xmin)
+ else if (nretain_conflict_info && can_advance_xmin)
advance_conflict_slot_xmin(xmin);
- else if (!retain_conflict_info)
+ else if (!nretain_conflict_info)
drop_conflict_slot_if_exists();
/* Switch back to original memory context. */
@@ -1418,6 +1443,31 @@ drop_conflict_slot_if_exists(void)
conflict_slot_dropped = true;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1461,7 +1511,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;
@@ -1538,6 +1588,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 99b101f8766..c97c6937a02 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -454,6 +454,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3866,7 +3868,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4142,6 +4145,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4282,6 +4289,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4363,6 +4377,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4388,7 +4421,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4408,12 +4444,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4421,9 +4466,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4456,6 +4556,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 668279cb4e8..bce4af1bdcd 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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 d3d28a263fa..846023894cc 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 c9ef5259b68..51c411da6c0 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v33-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v33-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From b3c112ff3dde08305eca415c48be990b9f16c1a7 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 3 Jun 2025 18:52:15 +0800
Subject: [PATCH v33 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 | 41 ++++++++++-----
src/backend/replication/logical/worker.c | 60 +++++++++++++++-------
3 files changed, 74 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 8712894a791..2f8dfb10cc6 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5419,7 +5419,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f5f3c87042b..d351d02d878 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -108,7 +108,7 @@ 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 create_conflict_slot_if_not_exists(bool recreate_if_invalid);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
static void invalidate_conflict_slot(void);
@@ -322,7 +322,7 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
bool is_tablesync_worker = (wtype == WORKERTYPE_TABLESYNC);
bool is_parallel_apply_worker = (wtype == WORKERTYPE_PARALLEL_APPLY);
bool retaining_conflict_info = (MyReplicationSlot &&
- MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
+ MyReplicationSlot->data.invalidated == RS_INVAL_NONE);
/*----------
* Sanity checks:
@@ -458,7 +458,7 @@ retry:
worker->oldest_nonremovable_xid = retaining_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
- worker->stop_conflict_info_retention = retaining_conflict_info;
+ worker->stop_conflict_info_retention = !retaining_conflict_info;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1206,7 +1206,7 @@ ApplyLauncherMain(Datum main_arg)
* prevent it from unnecessarily maintaining its
* oldest_nonremovable_xid.
*/
- create_conflict_slot_if_not_exists();
+ create_conflict_slot_if_not_exists(!sub->enabled);
}
if (!sub->enabled)
@@ -1259,9 +1259,14 @@ ApplyLauncherMain(Datum main_arg)
/*
* The worker has not yet started, so there is no valid
* non-removable transaction ID available for advancement.
+ * Additionally, recreate the slot to enable the new worker to
+ * resume retaining conflict information.
*/
if (sub->retainconflictinfo)
+ {
can_advance_xmin = false;
+ create_conflict_slot_if_not_exists(true);
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1344,21 +1349,31 @@ ApplyLauncherMain(Datum main_arg)
* conflict detection, if not yet.
*/
static void
-create_conflict_slot_if_not_exists(void)
+create_conflict_slot_if_not_exists(bool recreate_if_invalid)
{
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))
- {
+ /*
+ * Acquire the replication slot if it exists and hasn't been acquired yet.
+ */
+ if (!MyReplicationSlot &&
+ SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+
+ /* Drop the slot if it's invalidated and recreate_if_invalid is true */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE &&
+ recreate_if_invalid)
+ drop_conflict_slot_if_exists();
+
+ /* Return if a valid replication slot is already created and acquired */
+ if (MyReplicationSlot)
return;
- }
+ /*
+ * Create a new replication slot if none exists or the previous one was
+ * dropped.
+ */
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
RS_PERSISTENT, false, false, false);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c97c6937a02..a85858df977 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -482,6 +482,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4145,10 +4147,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4428,6 +4426,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to restarting
+ * the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4435,6 +4452,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4476,9 +4494,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4507,19 +4524,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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));
+ /*
+ * 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 = InvalidTransactionId;
- MyLogicalRepWorker->stop_conflict_info_retention = true;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
- /* Notify launcher to invalidate the conflict slot */
- ApplyLauncherWakeup();
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v33-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v33-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From fb6c312c710740745c6a1f30b25a4db27a5df7f6 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v33 6/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 164 ++++++++++++++++++++++-
1 file changed, 162 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..7706465b351 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 @@ $node_publisher->start;
# 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,164 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Disable failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = false)");
+ok( $stderr =~
+ /ERROR: cannot set option retain_conflict_info for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
On Wed, Jun 4, 2025 at 6:43 PM Zhijie Hou (Fujitsu) wrote:
On Mon, Jun 2, 2025 at 2:39 PM Amit Kapila wrote:
On Mon, May 26, 2025 at 12:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attaching the V32 patch set which addressed comments in [1]~[5].
Review comments: =============== * +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;The above code suggests that the launcher could compute a new xmin
that is less than slot's xmin. At first, this looks odd to me, but
IIUC, this can happen when the user toggles retain_conflict_info
flag at some random time when the launcher is trying to compute the
new xmin value for the slot. One of the possible combinations of
steps for thisrace could be as follows:
1. The subscriber has two subscriptions, A and B. Subscription A has
retain_conflict_info as true, and B has retain_conflict_info as
false 2. Say the launcher calls get_subscription_list(), and worker
A is alreadyalive.
3. Assuming the apply worker will restart on changing
retain_conflict_info, the user enables retain_conflict_info for subscription B.
4. The launcher processes the subscription B first in the first
cycle, and starts worker B. Say, worker B gets 759 as candidate_xid.
5. The launcher creates the conflict detection slot, xmin = 759 6.
Say a new txn happens, worker A gets 760 as candidate_xid and
updates it to oldest_nonremovable_xid.
7. The launcher processes the subscription A in the first cycle, and
the final xmin value is 760, because it only checks the
oldest_nonremovable_xid from worker A. The launcher then updates thevalue to slot.xmin.
8. In the next cycle, the launcher finds that worker B has an older
oldest_nonremovable_xid 759, so the minimal xid would now be 759.
The launher would have retreated the slot's xmin unless we had the
above check in the quoted code.I think the above race is possible because the system lets the
changed subscription values of a subscription take effect asynchronously by workers.
The one more similar race condition handled by the patch is as follows:* ... + * 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 + * w.r.t the transaction ID stored in the slot maintained by launcher. + See + * advance_conflict_slot_xmin. ... + FullTransactionId oldest_nonremovable_xid;This case can happen if the user disables and immediately enables
the retain_conflict_info option. In this case, the launcher may drop
the slot after noticing the disable. But the apply worker may not
notice the disable and it only notices that the retain_conflict_info
is still enabled, so it will keep maintaining
oldest_nonremovable_xid when the slotis not created.
It is okay to handle both the race conditions, but I am worried we
may miss some such race conditions which could lead to
difficult-to-find bugs. So, at least for the first version of this
option (aka for patches 0001 to 0003), we can add a condition that
allows us to change retain_conflict_info only on disabled
subscriptions. This will simplify thepatch.
Agreed.
We can make a separate patch to
allow changing retain_conflict_info option for enabled subscriptions.
That will make it easier to evaluate such race conditions and the
solutionsmore deeply.
I will prepare a separate patch soon.
Here is the V33 patch set which includes the following changes:
0001:
* Renaming and typo fixes based on Shveta's comments [1]
* Comment changes suggested by Amit [2]
* Changed oldest_nonremoable_xid from FullTransactionID to TransactionID.
* Code refactoring in drop_conflict_slot_if_exists()0002:
* Documentation updates suggested by Amit [2]
* Code modifications to adapt to TransactionID oldest_nonremoable_xid0003:
* Documentation improvements suggested by Shveta [3]
* Added restriction: disallow changing retain_conflict_info when sub
is enabled or worker is alive
One thing I forgot to mention. After adding the restriction, one race condition
still exists:
Consider an enabled subscription A with retain_conflict_info=false. If the
launcher has fetched the subscription info from get_subscription_list() but
hasn't yet started a worker for subscription A, and the user executes the
following DDLs[1]ALTER SUBSCRIPTION A DISABLE; ALTER SUBSCRIPTION A SET (retain_conflict_info=true); ALTER SUBSCRIPTION A ENABLE; during this period, it would create a race condition where
the worker starts maintaining oldest_nonremovable_xid while the slot hasn't
been created.
To handle this, I added a check in InitializeLogRepWorker() that restarts the
worker if the retain_conflict_info option changes during startup. This would
ensure the worker runs with the latest configuration. This is also similar to
the existing enable option check in that function.
[1]: ALTER SUBSCRIPTION A DISABLE; ALTER SUBSCRIPTION A SET (retain_conflict_info=true); ALTER SUBSCRIPTION A ENABLE;
ALTER SUBSCRIPTION A DISABLE;
ALTER SUBSCRIPTION A SET (retain_conflict_info=true);
ALTER SUBSCRIPTION A ENABLE;
0004:
* Simplified race condition handling due to the new restriction from
00030005:
* Code updates to accommodate both the TransactionID type for
oldest_nonremoable_xid and the new restriction from 00030006:
* New test case for the restriction introduced in 00030007:
No changes
Best Regards,
Hou zj
On Wed, Jun 4, 2025 at 4:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V33 patch set which includes the following changes:
Thank You for the patches, please find few comments for patch003:
1)
+ /*
+ * Skip the track_commit_timestamp check by passing it as
+ * true, since it has already been validated during CREATE
+ * SUBSCRIPTION and ALTER SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ true, opts.enabled);
+
Is there a special reason for disabling WARNING while enabling the
subscription? If rci subscription was created in disabled state and
track_commit_timestamp was enabled at that time, then there will be no
WARNING. But while enabling the sub at a later stage, it may be
possible that track_commit_timestamp is off but rci as ON.
2)
* The worker has not yet started, so there is no valid
* non-removable transaction ID available for advancement.
*/
+ if (sub->retainconflictinfo)
+ can_advance_xmin = false;
Shall we change comment to:
Only advance xmin when all workers for rci enabled subscriptions are
up and running.
3)
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
new_xmin));
+ if (!TransactionIdIsValid(new_xmin))
+ return;
a)
Why have we replaced Assert with 'if' check? In which scenario do we
expect new_xmin as Invalid here?
b)
Even if we have if-check, shall it come before :
Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
new_xmin));
4)
DisableSubscriptionAndExit:
+ /*
+ * Skip the track_commit_timestamp check by passing it as true, since it
+ * has already been validated during CREATE SUBSCRIPTION and ALTER
+ * SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true,
+ false);
This comment makes sense during alter-sub enable, here shall we change it to:
Skip the track_commit_timestamp check by passing it as true as it is
not needed to be checked during subscription-disable.
5)
postgres=# alter subscription sub3 set (retain_conflict_info=false);
ERROR: cannot set option retain_conflict_info for enabled subscription
Do we need this restriction during disable of rci as well?
6)
+ <para>
+ If the <link
linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot that was created
+ to retain conflict information will be dropped.
+ </para>
It will be good to mention the slot name as well.
7)
+ * 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.
*/
We shall rephrase the comment, it gives the feeling that a 'conflict
detection slot' is needed for each subscription.
thanks
Shveta
On Fri, Jun 6, 2025 at 1:49 PM shveta malik wrote:
On Wed, Jun 4, 2025 at 4:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V33 patch set which includes the following changes:
please find few comments for patch003:
Thanks for the comments!
1) + /* + * Skip the track_commit_timestamp check by passing it as + * true, since it has already been validated during CREATE + * SUBSCRIPTION and ALTER SUBSCRIPTION SET commands. + */ + CheckSubConflictInfoRetention(sub->retainconflictinfo, + true, opts.enabled); +Is there a special reason for disabling WARNING while enabling the
subscription? If rci subscription was created in disabled state and
track_commit_timestamp was enabled at that time, then there will be no
WARNING. But while enabling the sub at a later stage, it may be
possible that track_commit_timestamp is off but rci as ON.
I feel reporting a WARNING related to track_commit_timestamp during
subscription enable DDL is a bit unnatural, since it's not directly related to the
this DDL. Also, I think we do not intend to capture scenarios where
track_commit_timestamp is disabled afterwards.
2)
* The worker has not yet started, so there is no valid
* non-removable transaction ID available for advancement.
*/
+ if (sub->retainconflictinfo)
+ can_advance_xmin = false;Shall we change comment to:
Only advance xmin when all workers for rci enabled subscriptions are
up and running.
Adjusted according to your suggestion.
3)
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
new_xmin));+ if (!TransactionIdIsValid(new_xmin))
+ return;a)
Why have we replaced Assert with 'if' check? In which scenario do we
expect new_xmin as Invalid here?
I think it's not needed now, so removed.
4) DisableSubscriptionAndExit: + /* + * Skip the track_commit_timestamp check by passing it as true, since it + * has already been validated during CREATE SUBSCRIPTION and ALTER + * SUBSCRIPTION SET commands. + */ + CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true, + false);This comment makes sense during alter-sub enable, here shall we change it
to:
Skip the track_commit_timestamp check by passing it as true as it is
not needed to be checked during subscription-disable.
Changed.
5)
postgres=# alter subscription sub3 set (retain_conflict_info=false);
ERROR: cannot set option retain_conflict_info for enabled subscriptionDo we need this restriction during disable of rci as well?
I prefer to maintain the restriction on both enabling and disabling operations
for the sake of simplicity, since the primary aim of this restriction is to
keep the logic straightforward and eliminate the need to think and address all
potential race conditions. I think restricting only the enable operation is
also OK and would not introducing new issues, but it might be more prudent to
keep things simple in the first version. Once the main patches stabilize, we
can consider easing or removing the entire restriction.
6) + <para> + If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literalretain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription + has this option enabled, the additional replication slot that was created + to retain conflict information will be dropped. + </para>It will be good to mention the slot name as well.
Added.
7) + * 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. */We shall rephrase the comment, it gives the feeling that a 'conflict
detection slot' is needed for each subscription.
Right, changed.
Here is the V34 patch set which includes the following changes:
0001:
* pgindent
0002:
* pgindent
0003:
* pgindent
* Addressed above comments from Shvete
* Improved the comments atop of the new restrcition.
* Ensured that the worker restarts when the retain_conflict_info was enabled
during startup regardless of the existence of the slot.
In V33, we relied on the existence of slot to decide whether the worker needs
to restart on startup option change. But we found that even if the slot
exists when launching the apply worker with(retain_conflict_info=false), the
slot could be removed soon by the launcher since the launcher might find
there is no subscription that enables retain_conflict_info. So the worker
could start to maintain the oldest_nonremovable_xid when the slot is not yet
created.
0004:
* pgindent
* Fixed some inaccurate and wrong descriptions in the document.
0005:
* pgindent
0006:
* pgindent
0007:
* pgindent
0008:
* A new patch to remove the restriction on altering retain_conflict_info when
the subscription is enabled, and resolves race condition issues caused by the
new option value being asynchronously acknowledged by the launcher and apply
workers. It changed the oldest_nonremovable_xid to FullTransactionID so that
even if the warparound happens, we can correctly identity if the transaction ID
a old or new one. Additioanly, it adds a safeguard check when advancing
slot.xmin to prevent backward movement.
The 0008 is kept as .txt to prevent the BF failure from testing it at this stage.
Best Regards,
Hou zj
Attachments:
v34-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v34-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 33168e79a0b7377efb1867655eb891c0055567e8 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 3 Jun 2025 11:40:30 +0800
Subject: [PATCH v34 2/8] 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.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
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 | 170 +++++++++++++++++-
.../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, 236 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 021153b2a5f..20b74d92006 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,6 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c67688cbf5f..1fd0f908554 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index ed56df85ca7..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 9b155705dbb..c70ce91da40 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"
@@ -90,8 +91,12 @@ static dshash_table *last_start_times = NULL;
static bool on_commit_launcher_wakeup = false;
+/*
+ * Whether the slot used to retain dead tuples for conflict detection has been
+ * dropped.
+ */
+static bool conflict_slot_dropped = 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 +105,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(TransactionId new_xmin);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1114,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1147,6 +1158,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1166,15 +1179,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)
+ {
+ TransactionId 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 (!TransactionIdIsValid(nonremovable_xid))
+ can_advance_xmin = false;
+ else if (!TransactionIdIsValid(xmin) ||
+ TransactionIdPrecedes(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 +1261,16 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (sublist && can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ else if (!sublist)
+ drop_conflict_slot_if_exists();
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1234,6 +1298,108 @@ 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();
+
+ conflict_slot_dropped = false;
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
+ new_xmin));
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->data.xmin = 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)
+{
+ /*
+ * Avoid the overhead of scanning shared memory for a replication slot
+ * that is known to have been dropped.
+ */
+ if (conflict_slot_dropped)
+ return;
+
+ if (MyReplicationSlot)
+ ReplicationSlotDropAcquired();
+ else if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ ReplicationSlotDrop(CONFLICT_DETECTION_SLOT, true);
+
+ conflict_slot_dropped = 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 67655111875..85239f6c316 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 4cd9efb8c94..c864f7f52b0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4386,6 +4386,9 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
LSN_FORMAT_ARGS(rci_data->remote_lsn),
rci_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. We can even use
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 600b87fa9cb..668279cb4e8 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 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
--
2.30.0.windows.2
v34-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v34-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 2a99d4c8e4433d1a9d5b9795ab47ea43c52bdd3b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 4 Jun 2025 16:08:45 +0800
Subject: [PATCH v34 3/8] 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 | 11 +
doc/src/sgml/logical-replication.sgml | 6 +
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 35 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 210 ++++++++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 67 ++++--
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 79 ++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 53 ++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 4 +
src/include/replication/worker_internal.h | 3 +-
src/test/regress/expected/subscription.out | 179 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
25 files changed, 670 insertions(+), 140 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..b8f9bf573ea 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is still useful for conflict detection
+ is retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 686dd441d02..5073d31ca8e 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e6f1cffeda5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..be90088bcd0 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,41 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, an additional replication
+ slot named <quote><literal>pg_conflict_detection</literal></quote>
+ will be created on the subscriber to prevent the conflict information
+ from being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..810c459196f 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_pub_conflict_info_retention(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,10 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ opts.enabled);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1110,6 +1135,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 +1191,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 +1351,82 @@ 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;
+
+ /*
+ * Do not allow changing the option when the subscription
+ * is enabled and the apply worker is active, to prevent
+ * race conditions arising from the new option value being
+ * acknowledged asynchronously by the launcher and apply
+ * workers.
+ *
+ * Without the restriction, a race condition may arise
+ * when a user disables and immediately re-enables the
+ * retain_conflict_info option. In this case, the launcher
+ * might drop the slot upon noticing the disable action,
+ * while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option
+ * change. During this period, a transaction ID wraparound
+ * could falsely make this ID appear as if it originates
+ * from the future w.r.t the transaction ID stored in the
+ * slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Similarily, if the user enables retain_conflict_info
+ * concurrently with the launcher starting the worker, the
+ * apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update
+ * slot.xmin to a newer value than that maintained by the
+ * worker. In subsequent cycles, upon integrating the
+ * worker's oldest_nonremovable_xid, the launcher might
+ * detect a regression in the calculated xmin,
+ * necessitating additional handling.
+ *
+ * While defining oldest_nonremovable_xid using
+ * FullTransactionID and checks against backward shifts in
+ * slot.xmin could address these concerns, further
+ * research is needed to ensure that all race conditions
+ * are identified and addressed. Therefore, we prioritize
+ * simplicity by preventing these races from occurring.
+ *
+ * Note that we could restrict only the enabling of
+ * retain_conflict_info to avoid the race conditions
+ * described above, but we maintain the restriction for
+ * both enable and disable operations for consistency and
+ * simplicity.
+ */
+ if (sub->enabled)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot set option %s for enabled subscription",
+ "retain_conflict_info")));
+
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ track_commit_timestamp,
+ sub->enabled);
+
+ /*
+ * 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] =
@@ -1347,6 +1449,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Skip the track_commit_timestamp check by passing it as
+ * true, since it has already been validated during CREATE
+ * SUBSCRIPTION and ALTER SUBSCRIPTION SET commands.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ true, opts.enabled);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1465,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 +1487,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 +1693,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 +1710,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 +1726,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_pub_conflict_info_retention(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 +2329,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_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c70ce91da40..1bf1dce26d3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -92,8 +92,8 @@ static dshash_table *last_start_times = NULL;
static bool on_commit_launcher_wakeup = false;
/*
- * Whether the slot used to retain dead tuples for conflict detection has been
- * dropped.
+ * Whether the slot used to retain information useful for conflict detection
+ * has been dropped.
*/
static bool conflict_slot_dropped = false;
@@ -156,6 +156,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);
@@ -304,7 +305,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -323,10 +325,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -449,7 +454,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1159,6 +1166,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1180,16 +1188,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);
@@ -1199,10 +1217,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)
{
TransactionId nonremovable_xid;
@@ -1225,10 +1244,11 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * The worker has not yet started, so there is no valid
- * non-removable transaction ID available for advancement.
+ * Only advance xmin when all workers for subscriptions with
+ * retain_conflict_info enabled are running.
*/
- can_advance_xmin = false;
+ if (sub->retainconflictinfo)
+ can_advance_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1252,7 +1272,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1266,9 +1287,9 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed. Otherwise, drop the slot if we're no longer
* retaining information useful for conflict detection.
*/
- if (sublist && can_advance_xmin)
+ if (retain_conflict_info && can_advance_xmin)
advance_conflict_slot_xmin(xmin);
- else if (!sublist)
+ else if (!retain_conflict_info)
drop_conflict_slot_if_exists();
/* Switch back to original memory context. */
@@ -1299,7 +1320,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
@@ -1343,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(TransactionId new_xmin)
@@ -1379,8 +1400,8 @@ advance_conflict_slot_xmin(TransactionId 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/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c864f7f52b0..36d9142c7d3 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"
@@ -445,6 +446,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4116,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4136,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5255,28 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled at startup. The
+ * replication slot for conflict detection may not be created yet, or
+ * might soon be dropped as the launcher sees retain_conflict_info as
+ * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
+ * when the slot is absent or at risk of being dropped, a restart is
+ * initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5433,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check by passing it as true when
+ * disabling the worker due to an error, as verifying commit timestamps is
+ * unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true,
+ false);
+
proc_exit(0);
}
@@ -5754,3 +5803,29 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Give a warning if track_commit_timestamp is not enabled or if the
+ * subscription is disabled.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool track_commit_timestamp,
+ bool sub_enabled)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (!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 (!sub_enabled)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("information for detecting conflicts cannot be purged when the subscription is disabled"));
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 37432e66efd..13d57dd3b13 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 7417eab6aef..945b3fce670 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 940fc77fc2e..3a3c532db47 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();
}
@@ -2017,9 +2017,11 @@ 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 for each subscription and to
+ * create the conflict detection slot when any subscription has the
+ * retain_conflict_info option enabled.
*/
static void
check_new_cluster_subscription_configuration(void)
@@ -2027,6 +2029,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)
@@ -2052,6 +2056,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);
@@ -2114,6 +2143,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 24e0100c9f0..c86010c2b89 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 ec65ab79fec..cbf2703c190 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3728,8 +3729,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ae46fa50ce6 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,8 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool track_commit_timestamp,
+ bool sub_enabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index c9ef5259b68..576626c6557 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -256,7 +256,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..310eb362511 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,45 @@ 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: information for detecting conflicts cannot be purged when the subscription 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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v34-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v34-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 3c08c50c295e8e2fe3f3d5f2222265874bc9308a Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 6 Jun 2025 13:00:17 +0800
Subject: [PATCH v34 4/8] 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 | 43 ++++++
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 | 95 ++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 311 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20b74d92006..355a4ac2977 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5395,6 +5395,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 1bf1dce26d3..0d2fd4c9a7c 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;
@@ -108,7 +111,7 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
-
+static void invalidate_conflict_slot(void);
/*
* Load the list of subscriptions.
@@ -457,6 +460,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1166,7 +1171,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;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1195,7 +1201,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- retain_conflict_info = true;
+ nretain_conflict_info++;
can_advance_xmin &= sub->enabled;
/*
@@ -1221,22 +1227,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)
{
TransactionId 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 (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
can_advance_xmin = false;
- else if (!TransactionIdIsValid(xmin) ||
- TransactionIdPrecedes(nonremovable_xid, xmin))
+ else if (can_advance_xmin &&
+ (!TransactionIdIsValid(xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, xmin)))
xmin = nonremovable_xid;
}
@@ -1282,14 +1298,22 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Invalidate the conflict slot if all workers with
+ * retain_conflict_info enabled have stopped further conflict
+ * retention.
+ */
+ if (nstop_retention && nretain_conflict_info == nstop_retention)
+ invalidate_conflict_slot();
+
/*
* Maintain the xmin value of the replication slot for conflict
* detection if needed. Otherwise, drop the slot if we're no longer
* retaining information useful for conflict detection.
*/
- if (retain_conflict_info && can_advance_xmin)
+ else if (nretain_conflict_info && can_advance_xmin)
advance_conflict_slot_xmin(xmin);
- else if (!retain_conflict_info)
+ else if (!nretain_conflict_info)
drop_conflict_slot_if_exists();
/* Switch back to original memory context. */
@@ -1421,6 +1445,31 @@ drop_conflict_slot_if_exists(void)
conflict_slot_dropped = true;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1464,7 +1513,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;
@@ -1541,6 +1590,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 36d9142c7d3..60305103fbb 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -454,6 +454,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3866,7 +3868,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4142,6 +4145,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4282,6 +4289,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4363,6 +4377,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4388,7 +4421,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4408,12 +4444,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4421,9 +4466,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4456,6 +4556,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 668279cb4e8..bce4af1bdcd 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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 d3d28a263fa..846023894cc 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 576626c6557..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v34-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v34-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 1f64b5d17fd76acf6719c1070b882d450b306fa1 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 6 Jun 2025 10:43:02 +0800
Subject: [PATCH v34 5/8] 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 | 39 +++++++++----
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 75 insertions(+), 33 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 355a4ac2977..d97eef1216d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5421,7 +5421,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 0d2fd4c9a7c..1b78cb53161 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -108,7 +108,7 @@ 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 create_conflict_slot_if_not_exists(bool recreate_if_invalid);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
static void invalidate_conflict_slot(void);
@@ -1209,7 +1209,7 @@ ApplyLauncherMain(Datum main_arg)
* prevent it from unnecessarily maintaining its
* oldest_nonremovable_xid.
*/
- create_conflict_slot_if_not_exists();
+ create_conflict_slot_if_not_exists(!sub->enabled);
}
if (!sub->enabled)
@@ -1261,10 +1261,15 @@ ApplyLauncherMain(Datum main_arg)
/*
* Only advance xmin when all workers for subscriptions with
- * retain_conflict_info enabled are running.
+ * retain_conflict_info enabled are running. Additionally, recreate
+ * the slot if it was invalidated to enable the new worker to
+ * resume retaining conflict information.
*/
if (sub->retainconflictinfo)
+ {
can_advance_xmin = false;
+ create_conflict_slot_if_not_exists(true);
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1348,21 +1353,31 @@ ApplyLauncherMain(Datum main_arg)
* conflict detection, if not yet.
*/
static void
-create_conflict_slot_if_not_exists(void)
+create_conflict_slot_if_not_exists(bool recreate_if_invalid)
{
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))
- {
+ /*
+ * Acquire the replication slot if it exists and hasn't been acquired yet.
+ */
+ if (!MyReplicationSlot &&
+ SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+
+ /* Drop the slot if it's invalidated and recreate_if_invalid is true */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE &&
+ recreate_if_invalid)
+ drop_conflict_slot_if_exists();
+
+ /* Return if a valid replication slot is already created and acquired */
+ if (MyReplicationSlot)
return;
- }
+ /*
+ * Create a new replication slot if none exists or the previous one was
+ * dropped.
+ */
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
RS_PERSISTENT, false, false, false);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 60305103fbb..f3c21434f79 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -482,6 +482,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4145,10 +4147,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4428,6 +4426,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4435,6 +4452,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4476,9 +4494,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4507,19 +4524,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v34-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v34-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 25a3fd98a68d71a4cc6558a76aad5b5c7c553cea Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v34 6/8] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 164 ++++++++++++++++++++++-
1 file changed, 162 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..7706465b351 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 @@ $node_publisher->start;
# 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,164 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Disable failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = false)");
+ok( $stderr =~
+ /ERROR: cannot set option retain_conflict_info for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v34-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v34-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 18c6ea05947e5b96f9e2a35dc6ce8271f022574f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 4 Jun 2025 16:40:43 +0800
Subject: [PATCH v34 7/8] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 10 +-
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 +++-
14 files changed, 268 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index b8f9bf573ea..24a2e9114d9 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 5073d31ca8e..f75ae3f01f2 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index be90088bcd0..778b51e335c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,10 +445,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
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
- <literal>false</literal>. If set to true, an additional replication
- slot named <quote><literal>pg_conflict_detection</literal></quote>
- will be created on the subscriber to prevent the conflict information
- from being removed.
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
</para>
<para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 f3c21434f79..92595b22d40 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2815,17 +2815,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. */
@@ -3163,18 +3174,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 1c12ddbae49..2a084d3f3f0 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 846023894cc..6b375f555a8 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 7706465b351..65c98c44cac 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -239,6 +242,8 @@ ok( $stderr =~
###############################################################################
# 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
@@ -267,10 +272,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v34-0008-Allow-altering-retain_conflict_info-for-enabled-.patch.txttext/plain; name=v34-0008-Allow-altering-retain_conflict_info-for-enabled-.patch.txtDownload
From e07f2b6f86477629dfd112cff1e09a5f6c804c9f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 6 Jun 2025 17:45:38 +0800
Subject: [PATCH v34 8/8] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 55 -------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 69 ++++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 79 ++++++++-----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 14 +---
8 files changed, 110 insertions(+), 147 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e6f1cffeda5..47efa5d86a5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 810c459196f..aa353486a4f 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1357,61 +1357,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- /*
- * Do not allow changing the option when the subscription
- * is enabled and the apply worker is active, to prevent
- * race conditions arising from the new option value being
- * acknowledged asynchronously by the launcher and apply
- * workers.
- *
- * Without the restriction, a race condition may arise
- * when a user disables and immediately re-enables the
- * retain_conflict_info option. In this case, the launcher
- * might drop the slot upon noticing the disable action,
- * while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option
- * change. During this period, a transaction ID wraparound
- * could falsely make this ID appear as if it originates
- * from the future w.r.t the transaction ID stored in the
- * slot maintained by launcher. See
- * advance_conflict_slot_xmin.
- *
- * Similarily, if the user enables retain_conflict_info
- * concurrently with the launcher starting the worker, the
- * apply worker may start calculating
- * oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update
- * slot.xmin to a newer value than that maintained by the
- * worker. In subsequent cycles, upon integrating the
- * worker's oldest_nonremovable_xid, the launcher might
- * detect a regression in the calculated xmin,
- * necessitating additional handling.
- *
- * While defining oldest_nonremovable_xid using
- * FullTransactionID and checks against backward shifts in
- * slot.xmin could address these concerns, further
- * research is needed to ensure that all race conditions
- * are identified and addressed. Therefore, we prioritize
- * simplicity by preventing these races from occurring.
- *
- * Note that we could restrict only the enabling of
- * retain_conflict_info to avoid the race conditions
- * described above, but we maintain the restriction for
- * both enable and disable operations for consistency and
- * simplicity.
- */
- if (sub->enabled)
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot set option %s for enabled subscription",
- "retain_conflict_info")));
-
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
CheckSubConflictInfoRetention(opts.retainconflictinfo,
track_commit_timestamp,
sub->enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 1b78cb53161..2b60da2cd19 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -109,7 +109,7 @@ 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(bool recreate_if_invalid);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
static void invalidate_conflict_slot(void);
@@ -308,8 +308,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,13 +327,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -457,10 +453,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1173,7 +1167,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
int nretain_conflict_info = 0;
int nstop_retention = 0;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1229,7 +1223,7 @@ ApplyLauncherMain(Datum main_arg)
*/
if (sub->retainconflictinfo)
{
- TransactionId nonremovable_xid;
+ FullTransactionId nonremovable_xid;
bool stop_conflict_info_retention;
SpinLockAcquire(&w->relmutex);
@@ -1248,11 +1242,11 @@ ApplyLauncherMain(Datum main_arg)
* Stop advancing xmin if an invalid non-removable
* transaction ID is found, otherwise update xmin.
*/
- else if (!TransactionIdIsValid(nonremovable_xid))
+ else if (!FullTransactionIdIsValid(nonremovable_xid))
can_advance_xmin = false;
else if (can_advance_xmin &&
- (!TransactionIdIsValid(xmin) ||
- TransactionIdPrecedes(nonremovable_xid, xmin)))
+ (!FullTransactionIdIsValid(xmin) ||
+ FullTransactionIdPrecedes(nonremovable_xid, xmin)))
xmin = nonremovable_xid;
}
@@ -1293,8 +1287,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1406,15 +1399,47 @@ create_conflict_slot_if_not_exists(bool recreate_if_invalid)
* information useful for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
- new_xmin));
+ 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
/* first write new xmin to disk, so we know what's up after a crash */
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 92595b22d40..3859a056b14 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -308,8 +308,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -482,8 +482,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4205,6 +4203,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4232,17 +4232,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4250,7 +4253,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4359,7 +4362,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4451,22 +4454,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4479,7 +4476,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4505,7 +4502,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4524,7 +4521,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4559,7 +4556,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4716,6 +4713,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 was changed. 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 ||
@@ -4725,7 +4731,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,
@@ -5405,28 +5412,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled at startup. The
- * replication slot for conflict detection may not be created yet, or
- * might soon be dropped as the launcher sees retain_conflict_info as
- * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
- * when the slot is absent or at risk of being dropped, a restart is
- * initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 65c98c44cac..4eacc0c089a 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -227,18 +227,6 @@ $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');
-##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
-##################################################
-
-# Disable failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = false)");
-ok( $stderr =~
- /ERROR: cannot set option retain_conflict_info for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
###############################################################################
# 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.
@@ -264,7 +252,7 @@ $node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
$node_A->wait_for_catchup($subname_BA);
-($cmdret, $stdout, $stderr) = $node_A->psql(
+my ($cmdret, $stdout, $stderr) = $node_A->psql(
'postgres', qq(VACUUM (verbose) public.tab;)
);
--
2.30.0.windows.2
v34-0001-Maintain-the-oldest-non-removable-transaction-ID.patchapplication/octet-stream; name=v34-0001-Maintain-the-oldest-non-removable-transaction-ID.patchDownload
From 366ac6e565bb68eb8d64fc04806f2af707a76a65 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v34 1/8] Maintain the oldest non-removable transaction 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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these 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 | 86 ++++
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 520 ++++++++++++++++++++-
src/backend/replication/walsender.c | 56 +++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/worker_internal.h | 11 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
12 files changed, 724 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..ed56df85ca7 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2643,6 +2643,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2846,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 2e67e998adb..fa3f5a4315c 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1459,10 +1459,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1565,7 +1571,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1914859b2ee..5f19981f9c3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 10677da56b2..9b155705dbb 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
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 a23262957ac..4cd9efb8c94 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,406 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_data->candidate_xid);
+
+ /*
+ * Reset all data fields except those used to determine the timing for the
+ * next round of transaction ID advancement. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_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 f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..c9ef5259b68 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,17 @@ 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 a8346cda633..b77253e35e4 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2554,6 +2554,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
On Wed, Jun 4, 2025 at 4:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V33 patch set which includes the following changes:
Few comments:
1.
+ if (sub->enabled)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot set option %s for enabled subscription",
+ "retain_conflict_info")));
Isn't it better to call CheckAlterSubOption() for this check, as we do
for failover and two_phase options?
2.
postgres=# Alter subscription sub1 set (retain_conflict_info=true);
ERROR: cannot set option retain_conflict_info for enabled subscription
postgres=# Alter subscription sub1 disable;
ALTER SUBSCRIPTION
postgres=# Alter subscription sub1 set (retain_conflict_info=true);
WARNING: information for detecting conflicts cannot be purged when
the subscription is disabled
ALTER SUBSCRIPTION
The above looks odd to me because first we didn't allow setting the
option for enabled subscription, and then when the user disabled the
subscription, a WARNING is issued. Isn't it better to give NOTICE
like: "enable the subscription to avoid accumulating deleted rows for
detecting conflicts" in the above case?
Also in this,
postgres=# Alter subscription sub1 set (retain_conflict_info=true);
WARNING: information for detecting conflicts cannot be fully retained
when "track_commit_timestamp" is disabled
WARNING: information for detecting conflicts cannot be purged when
the subscription is disabled
ALTER SUBSCRIPTION
What do we mean by this WARNING message? If track_commit_timestamp is
not enabled, we won't be able to detect certain conflicts, including
update_delete, but how can it lead to not retaining information
required for conflict detection? BTW, shall we consider giving ERROR
instead of WARNING for this case because without
track_commit_timestamp, there is no benefit in retaining deleted rows?
If we just want to make the user aware to enable
track_commit_timestamp to detect conflicts, then we can even consider
making this a NOTICE.
postgres=# Alter subscription sub1 Enable;
ALTER SUBSCRIPTION
postgres=# Alter subscription sub1 set (retain_conflict_info=false);
ERROR: cannot set option retain_conflict_info for enabled subscription
postgres=# Alter subscription sub1 disable;
WARNING: information for detecting conflicts cannot be purged when
the subscription is disabled
ALTER SUBSCRIPTION
Here, we should have a WARNING like: "deleted rows to detect conflicts
would not be removed till the subscription is enabled"; this should be
followed by errdetail like: "Consider setting retain_conflict_info to
false."
--
With Regards,
Amit Kapila.
On Fri, Jun 6, 2025 at 7:34 PM Amit Kapila wrote:
On Wed, Jun 4, 2025 at 4:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V33 patch set which includes the following changes:
Few comments: 1. + if (sub->enabled) + ereport(ERROR, + (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("cannot set option %s for enabled subscription", + "retain_conflict_info")));Isn't it better to call CheckAlterSubOption() for this check, as we do
for failover and two_phase options?
Moved.
2.
postgres=# Alter subscription sub1 set (retain_conflict_info=true);
ERROR: cannot set option retain_conflict_info for enabled subscription
postgres=# Alter subscription sub1 disable;
ALTER SUBSCRIPTION
postgres=# Alter subscription sub1 set (retain_conflict_info=true);
WARNING: information for detecting conflicts cannot be purged when
the subscription is disabled
ALTER SUBSCRIPTIONThe above looks odd to me because first we didn't allow setting the
option for enabled subscription, and then when the user disabled the
subscription, a WARNING is issued. Isn't it better to give NOTICE
like: "enable the subscription to avoid accumulating deleted rows for
detecting conflicts" in the above case?
Yes, a NOTICE would be better.
I think we normally only describe the current situation of the operation in a
NOTICE message, and the suggested message sounds like a hint.
So I used the following message:
"deleted rows will continue to accumulate for detecting conflicts until the subscription is enabled"
Also in this,
postgres=# Alter subscription sub1 set (retain_conflict_info=true);
WARNING: information for detecting conflicts cannot be fully retained
when "track_commit_timestamp" is disabled
WARNING: information for detecting conflicts cannot be purged when
the subscription is disabled
ALTER SUBSCRIPTIONWhat do we mean by this WARNING message? If track_commit_timestamp is
not enabled, we won't be able to detect certain conflicts, including
update_delete, but how can it lead to not retaining information
required for conflict detection? BTW, shall we consider giving ERROR
instead of WARNING for this case because without
track_commit_timestamp, there is no benefit in retaining deleted rows?
If we just want to make the user aware to enable
track_commit_timestamp to detect conflicts, then we can even consider
making this a NOTICE.
I think it's an unexpected case that track_commit_timestamp is not enabled, so
NOTICE may not be appropriate. Giving ERROR is also OK, but since user can
change the track_commit_timestamp setting at anytime after creating/modifying a
subscription, we can't catch all cases, so we considered simply issuing a
warning directly and document this case.
postgres=# Alter subscription sub1 Enable;
ALTER SUBSCRIPTION
postgres=# Alter subscription sub1 set (retain_conflict_info=false);
ERROR: cannot set option retain_conflict_info for enabled subscription
postgres=# Alter subscription sub1 disable;
WARNING: information for detecting conflicts cannot be purged when
the subscription is disabled
ALTER SUBSCRIPTIONHere, we should have a WARNING like: "deleted rows to detect conflicts
would not be removed till the subscription is enabled"; this should be
followed by errdetail like: "Consider setting retain_conflict_info to
false."
Changed as suggested.
Here is the V35 patch set which includes the following changes:
0001:
No change.
0002:
* Added an errdetail for reserved slot name error per off-list discussion with Shveta.
* Moves the codes in launcher's foreach loop to a new function to improve the readability.
0003:
* Addressed all above comments sent by Amit.
* Adjusted some comments per off-list discussion with Amit.
* Check track_commit_timestamp when enabling the subscription. This is to avoid
passing track_commit_timestamp as a parameter to the check function.
0004:
Rebased
0005:
Rebased
0006:
Rebased
0007:
Rebased
0008:
Rebased
Best Regards,
Hou zj
Attachments:
v35-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v35-0003-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From f8bede65f9d75875821e6aaf577a47b060560d5f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 10 Jun 2025 13:48:16 +0800
Subject: [PATCH v35 3/3] 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 | 11 +
doc/src/sgml/logical-replication.sgml | 6 +
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 35 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 201 ++++++++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 48 +++--
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 85 +++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 53 ++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 3 +
src/include/replication/worker_internal.h | 3 +-
src/test/regress/expected/subscription.out | 181 +++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
25 files changed, 655 insertions(+), 134 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..b8f9bf573ea 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is still useful for conflict detection
+ is retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 686dd441d02..5073d31ca8e 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e6f1cffeda5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..be90088bcd0 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,41 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, an additional replication
+ slot named <quote><literal>pg_conflict_detection</literal></quote>
+ will be created on the subscriber to prevent the conflict information
+ from being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..5bcc171ca5b 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_pub_conflict_info_retention(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,9 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ !opts.enabled || !opts.connect);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +690,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 +746,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1051,7 +1075,8 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1059,6 +1084,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
+ * Do not allow changing the retain_conflict_info option when the
+ * subscription is enabled or the apply worker is active, to prevent race
+ * conditions arising from the new option value being acknowledged
+ * asynchronously by the launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
+ *
* Note that two_phase is enabled (aka changed from 'false' to 'true') on
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
@@ -1110,6 +1170,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 +1226,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 +1386,41 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Note that workers may still survive even if the
+ * subscription has been disabled.
+ *
+ * Ensure workers have already been exited to avoid the
+ * race conditions as described in CheckAlterSubOption().
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ false);
+
+ /*
+ * 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] =
@@ -1347,6 +1443,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ !opts.enabled);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1454,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 +1476,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 +1682,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 +1699,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 +1715,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_pub_conflict_info_retention(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 +2318,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_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 9b12b2900e6..494b8de9ef9 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -92,8 +92,8 @@ static dshash_table *last_start_times = NULL;
static bool on_commit_launcher_wakeup = false;
/*
- * Whether the slot used to retain dead tuples for conflict detection has been
- * dropped.
+ * Whether the slot used to retain information useful for conflict detection
+ * has been dropped.
*/
static bool conflict_slot_dropped = false;
@@ -106,6 +106,7 @@ static void logicalrep_launcher_attach_dshmem(void);
static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
TransactionId *xmin,
bool *can_advance_xmin);
static void create_conflict_slot_if_not_exists(void);
@@ -159,6 +160,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);
@@ -307,7 +309,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -326,10 +329,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -452,7 +458,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1162,6 +1170,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1182,9 +1191,12 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
{
- compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
}
@@ -1192,7 +1204,8 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
- compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
if (w != NULL)
continue; /* worker is running already */
@@ -1219,7 +1232,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1233,7 +1247,7 @@ ApplyLauncherMain(Datum main_arg)
* detection if needed. Otherwise, drop the slot if we're no longer
* retaining information useful for conflict detection.
*/
- if (!sublist)
+ if (!retain_conflict_info)
drop_conflict_slot_if_exists();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1266,17 +1280,18 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Compute the minimum non-removable transaction ID from all apply workers.
- * Store the result in *xmin.
+ * Compute the minimum non-removable transaction ID from all apply workers for
+ * subscriptions with retain_conflict_info enabled. Store the result in *xmin.
*
* If the slot cannot be advanced during this cycle, due to either a disabled
* subscription or an inactive worker, *can_advance_xmin is set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
bool *can_advance_xmin)
{
- if (!*can_advance_xmin)
+ if (!retain_conflict_info || !*can_advance_xmin)
return;
if (worker)
@@ -1319,7 +1334,8 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
create_conflict_slot_if_not_exists();
/*
- * Only collect xmin when all workers for subscriptions are running.
+ * Only collect xmin when all workers for subscriptions with
+ * retain_conflict_info enabled are running.
*/
*can_advance_xmin = false;
}
@@ -1370,7 +1386,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(TransactionId new_xmin)
@@ -1406,8 +1422,8 @@ advance_conflict_slot_xmin(TransactionId 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/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c864f7f52b0..48eb387d41b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -147,6 +147,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -173,6 +174,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"
@@ -445,6 +447,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4117,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4137,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5256,28 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled at startup. The
+ * replication slot for conflict detection may not be created yet, or
+ * might soon be dropped as the launcher sees retain_conflict_info as
+ * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
+ * when the slot is absent or at risk of being dropped, a restart is
+ * initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5434,8 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, true);
+
proc_exit(0);
}
@@ -5754,3 +5798,40 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue a warning if track_commit_timestamp is not enabled.
+ *
+ * Issue a warning if the subscription is being disabled.
+ *
+ * Provide a notice if retain_conflict_info is enabled for a disabled
+ * subscription, reminding the user to enable the subscription to prevent the
+ * accumulation of dead tuples. A warning is not issued since
+ * retain_conflict_info can be altered only for disabled subscriptions.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool disabling_sub)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (!track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (disabling_sub)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
+ else
+ ereport(NOTICE,
+ errmsg("deleted rows will continue to accumulate for detecting conflicts until the subscription is enabled"));
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 37432e66efd..13d57dd3b13 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 7417eab6aef..945b3fce670 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 940fc77fc2e..3a3c532db47 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();
}
@@ -2017,9 +2017,11 @@ 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 for each subscription and to
+ * create the conflict detection slot when any subscription has the
+ * retain_conflict_info option enabled.
*/
static void
check_new_cluster_subscription_configuration(void)
@@ -2027,6 +2029,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)
@@ -2052,6 +2056,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);
@@ -2114,6 +2143,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 24e0100c9f0..c86010c2b89 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 ec65ab79fec..cbf2703c190 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3728,8 +3729,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..15039811de6 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,7 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool disabling_sub);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index c9ef5259b68..576626c6557 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -256,7 +256,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..373a8484f24 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,47 @@ 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: commit timestamp and origin data required for detecting conflicts won't be retained
+HINT: Consider setting "track_commit_timestamp" to true.
+WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled
+HINT: Consider setting retain_conflict_info to false.
+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 007c9e70374..c65397e5ac6 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;
--
2.30.0.windows.2
v35-0001-Maintain-the-oldest-non-removable-transaction-ID.patchapplication/octet-stream; name=v35-0001-Maintain-the-oldest-non-removable-transaction-ID.patchDownload
From ac9b77eed6caec6207d30eb53f202798e845355b Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v35 1/3] Maintain the oldest non-removable transaction 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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these 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 | 86 ++++
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/replication/logical/launcher.c | 1 +
src/backend/replication/logical/worker.c | 520 ++++++++++++++++++++-
src/backend/replication/walsender.c | 56 +++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/worker_internal.h | 11 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
12 files changed, 724 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..ed56df85ca7 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2643,6 +2643,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2846,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1914859b2ee..5f19981f9c3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 1c3c051403d..67bacc0f28c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -441,6 +441,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
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 a23262957ac..4cd9efb8c94 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,406 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_data->candidate_xid);
+
+ /*
+ * Reset all data fields except those used to determine the timing for the
+ * next round of transaction ID advancement. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_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 f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..c9ef5259b68 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,17 @@ 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 a8346cda633..b77253e35e4 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2554,6 +2554,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v35-0002-Maintain-the-replication-slot-in-logical-launche.patchapplication/octet-stream; name=v35-0002-Maintain-the-replication-slot-in-logical-launche.patchDownload
From 69781f596d88e28ebad1cafbf10ec1a562bbfd52 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 3 Jun 2025 11:40:30 +0800
Subject: [PATCH v35 2/3] 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.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
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 | 197 +++++++++++++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/worker.c | 3 +
src/backend/replication/slot.c | 36 +++-
src/include/replication/logicallauncher.h | 1 +
src/include/replication/slot.h | 11 +-
12 files changed, 265 insertions(+), 13 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 021153b2a5f..20b74d92006 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,6 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c67688cbf5f..1fd0f908554 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index ed56df85ca7..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 67bacc0f28c..9b12b2900e6 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"
@@ -90,8 +91,12 @@ static dshash_table *last_start_times = NULL;
static bool on_commit_launcher_wakeup = false;
+/*
+ * Whether the slot used to retain dead tuples for conflict detection has been
+ * dropped.
+ */
+static bool conflict_slot_dropped = 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 +105,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static void create_conflict_slot_if_not_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void drop_conflict_slot_if_exists(void);
/*
@@ -1106,7 +1117,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1147,6 +1161,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1167,12 +1183,17 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1207,6 +1228,16 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (!sublist)
+ drop_conflict_slot_if_exists();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1234,6 +1265,168 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Compute the minimum non-removable transaction ID from all apply workers.
+ * Store the result in *xmin.
+ *
+ * If the slot cannot be advanced during this cycle, due to either a disabled
+ * subscription or an inactive worker, *can_advance_xmin is set to false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * Assume the replication slot for conflict detection is created
+ * before the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /*
+ * Stop advancing xmin if an invalid non-removable transaction ID is
+ * found, otherwise update xmin.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ *can_advance_xmin = false;
+ else if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information (e.g., dead tuples,
+ * commit timestamps, and origins) useful for conflict detection if
+ * any subscription requests it.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid. It is
+ * created even for a disabled subscription to ensure information is
+ * available for detecting conflicts during the application of remote
+ * changes that occur before the subscription is enabled.
+ */
+ create_conflict_slot_if_not_exists();
+
+ /*
+ * Only collect xmin when all workers for subscriptions are running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Create and acquire the replication slot used to retain information 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();
+
+ conflict_slot_dropped = false;
+}
+
+/*
+ * Attempt to advance the xmin value of the replication slot used to retain
+ * dead tuples for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
+ new_xmin));
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->data.xmin = 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)
+{
+ /*
+ * Avoid the overhead of scanning shared memory for a replication slot
+ * that is known to have been dropped.
+ */
+ if (conflict_slot_dropped)
+ return;
+
+ if (MyReplicationSlot)
+ ReplicationSlotDropAcquired();
+ else if (SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ ReplicationSlotDrop(CONFLICT_DETECTION_SLOT, true);
+
+ conflict_slot_dropped = 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 67655111875..85239f6c316 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 4cd9efb8c94..c864f7f52b0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4386,6 +4386,9 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
LSN_FORMAT_ARGS(rci_data->remote_lsn),
rci_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. We can even use
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 600b87fa9cb..41adf71c4c8 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,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 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
--
2.30.0.windows.2
v35-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v35-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From ec2d4a08e073b4993ca85e03b465c8b7636f6f44 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 10 Jun 2025 13:14:08 +0800
Subject: [PATCH v35 4/8] 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 | 43 ++++++
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 | 106 +++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 324 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20b74d92006..355a4ac2977 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5395,6 +5395,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 e5feb249512..611f58b237d 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;
@@ -108,11 +111,12 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
-
+static void invalidate_conflict_slot(void);
/*
* Load the list of subscriptions.
@@ -461,6 +465,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1171,6 +1177,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1196,7 +1203,8 @@ ApplyLauncherMain(Datum main_arg)
if (!sub->enabled)
{
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1205,7 +1213,7 @@ ApplyLauncherMain(Datum main_arg)
LWLockRelease(LogicalRepWorkerLock);
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1242,12 +1250,19 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ */
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+
/*
* Maintain the xmin value of the replication slot for conflict
* detection if needed. Otherwise, drop the slot if we're no longer
* retaining information useful for conflict detection.
*/
- if (!retain_conflict_info)
+ else if (!retain_conflict_info)
drop_conflict_slot_if_exists();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1286,17 +1301,21 @@ ApplyLauncherMain(Datum main_arg)
*
* If the slot cannot be advanced during this cycle, due to either a disabled
* subscription or an inactive worker, *can_advance_xmin is set to false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1307,8 +1326,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
/*
* Stop advancing xmin if an invalid non-removable transaction ID is
* found, otherwise update xmin.
@@ -1339,6 +1372,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* retain_conflict_info enabled are running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1444,6 +1483,31 @@ drop_conflict_slot_if_exists(void)
conflict_slot_dropped = true;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1487,7 +1551,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;
@@ -1564,6 +1628,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 48eb387d41b..ebf0f007e5a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -455,6 +455,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3867,7 +3869,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4143,6 +4146,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4290,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4364,6 +4378,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4422,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4445,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4467,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4557,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 41adf71c4c8..0d52242fc89 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"},
};
/*
@@ -1612,6 +1613,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();
}
@@ -1730,6 +1736,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;
}
@@ -1997,6 +2009,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 d3d28a263fa..846023894cc 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 576626c6557..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v35-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v35-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From c4ca4ed73de7c257f6fc5ed929ab4ddbc5609aea Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 9 Jun 2025 19:13:55 +0800
Subject: [PATCH v35 5/8] 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 | 8 +++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 56 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 355a4ac2977..d97eef1216d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5421,7 +5421,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 611f58b237d..d1ecb6fb780 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1401,6 +1401,14 @@ create_conflict_slot_if_not_exists(void)
return;
}
+ /* Return if a valid replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ /*
+ * Create a new replication slot if none exists or the previous one was
+ * dropped.
+ */
ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
RS_PERSISTENT, false, false, false);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ebf0f007e5a..1c968b13d2a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -483,6 +483,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4146,10 +4148,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4429,6 +4427,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4436,6 +4453,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4477,9 +4495,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4508,19 +4525,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v35-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v35-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 72430a7141f04a365f71542a31b4a4215be31c04 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v35 6/8] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 164 ++++++++++++++++++++++-
1 file changed, 162 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..5bb269fa30f 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 @@ $node_publisher->start;
# 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,164 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Disable failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = false)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v35-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v35-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 357fa12c1e2897af27b6b3770064f0b6f3edf71a Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 4 Jun 2025 16:40:43 +0800
Subject: [PATCH v35 7/8] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 10 +-
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 +++-
14 files changed, 268 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index b8f9bf573ea..24a2e9114d9 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 5073d31ca8e..f75ae3f01f2 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index be90088bcd0..778b51e335c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,10 +445,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
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
- <literal>false</literal>. If set to true, an additional replication
- slot named <quote><literal>pg_conflict_detection</literal></quote>
- will be created on the subscriber to prevent the conflict information
- from being removed.
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
</para>
<para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 1c968b13d2a..7083341f35d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2816,17 +2816,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. */
@@ -3164,18 +3175,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 1c12ddbae49..2a084d3f3f0 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 846023894cc..6b375f555a8 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 5bb269fa30f..c5a3d570e0d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -239,6 +242,8 @@ ok( $stderr =~
###############################################################################
# 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
@@ -267,10 +272,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v35-0008-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v35-0008-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 43207bf14c70ef793ade718fae37ae95334ee47c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 10 Jun 2025 13:18:19 +0800
Subject: [PATCH v35 8/8] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 53 +------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 79 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 79 ++++++++-----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 14 +---
8 files changed, 113 insertions(+), 152 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e6f1cffeda5..47efa5d86a5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 2dbdd5aa375..2d6ec697607 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1075,8 +1075,7 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1084,41 +1083,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
- * Do not allow changing the retain_conflict_info option when the
- * subscription is enabled or the apply worker is active, to prevent race
- * conditions arising from the new option value being acknowledged
- * asynchronously by the launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
- *
* Note that two_phase is enabled (aka changed from 'false' to 'true') on
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
@@ -1392,21 +1356,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Note that workers may still survive even if the
- * subscription has been disabled.
- *
- * Ensure workers have already been exited to avoid the
- * race conditions as described in CheckAlterSubOption().
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
CheckSubConflictInfoRetention(opts.retainconflictinfo,
false);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index d1ecb6fb780..38837308ac3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -110,11 +110,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static void create_conflict_slot_if_not_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void drop_conflict_slot_if_exists(void);
static void invalidate_conflict_slot(void);
@@ -313,8 +313,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -333,13 +332,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -462,10 +458,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1178,7 +1172,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1240,8 +1234,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1307,7 +1300,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1316,13 +1309,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * Assume the replication slot for conflict detection is created
- * before the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1346,10 +1333,10 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* Stop advancing xmin if an invalid non-removable transaction ID is
* found, otherwise update xmin.
*/
- if (!TransactionIdIsValid(nonremovable_xid))
+ if (!FullTransactionIdIsValid(nonremovable_xid))
*can_advance_xmin = false;
- else if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ else if (!FullTransactionIdIsValid(*xmin) ||
+ FullTransactionIdPrecedes(nonremovable_xid, *xmin))
*xmin = nonremovable_xid;
}
else
@@ -1437,15 +1424,47 @@ create_conflict_slot_if_not_exists(void)
* information useful for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
- new_xmin));
+ 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
/* first write new xmin to disk, so we know what's up after a crash */
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 7083341f35d..88daa575a9e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,8 +309,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -483,8 +483,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4206,6 +4204,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4233,17 +4233,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4251,7 +4254,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4360,7 +4363,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4452,22 +4455,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4480,7 +4477,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4506,7 +4503,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4525,7 +4522,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4560,7 +4557,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4717,6 +4714,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 was changed. 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 ||
@@ -4726,7 +4732,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,
@@ -5406,28 +5413,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled at startup. The
- * replication slot for conflict detection may not be created yet, or
- * might soon be dropped as the launcher sees retain_conflict_info as
- * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
- * when the slot is absent or at risk of being dropped, a restart is
- * initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index c5a3d570e0d..4eacc0c089a 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -227,18 +227,6 @@ $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');
-##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
-##################################################
-
-# Disable failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = false)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
###############################################################################
# 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.
@@ -264,7 +252,7 @@ $node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
$node_A->wait_for_catchup($subname_BA);
-($cmdret, $stdout, $stderr) = $node_A->psql(
+my ($cmdret, $stdout, $stderr) = $node_A->psql(
'postgres', qq(VACUUM (verbose) public.tab;)
);
--
2.30.0.windows.2
On Tue, Jun 10, 2025 at 11:55 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V35 patch set which includes the following changes:
Few minor comments:
===================
1.
+Â * Check if the subscriber's configuration is adequate to enable the
+Â * retain_conflict_info option.
I see some funny characters in patch 0003.
2.
+static void
+drop_conflict_slot_if_exists(void)
+{
+ /*
+ * Avoid the overhead of scanning shared memory for a replication slot
+ * that is known to have been dropped.
+ */
+ if (conflict_slot_dropped)
+ return;
This new variable used here looks odd to me. Do you think we can avoid this?
Apart from this, I have made a number of cosmetic changes in the
attached. Kindly include these in the next version, if these look okay
to you. Also, I think we can combine 0001 and 0002 at this stage, as
both are looking in good shape now.
--
With Regards,
Amit Kapila.
Attachments:
v35_amit_1.txttext/plain; charset=US-ASCII; name=v35_amit_1.txtDownload
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 494b8de9ef9..74facd24a61 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1195,6 +1195,13 @@ ApplyLauncherMain(Datum main_arg)
if (!sub->enabled)
{
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
&xmin, &can_advance_xmin);
continue;
@@ -1204,6 +1211,10 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
&can_advance_xmin);
@@ -1243,9 +1254,10 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Maintain the xmin value of the replication slot for conflict
- * detection if needed. Otherwise, drop the slot if we're no longer
- * retaining information useful for conflict detection.
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (!retain_conflict_info)
drop_conflict_slot_if_exists();
@@ -1280,11 +1292,13 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Compute the minimum non-removable transaction ID from all apply workers for
- * subscriptions with retain_conflict_info enabled. Store the result in *xmin.
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
*
- * If the slot cannot be advanced during this cycle, due to either a disabled
- * subscription or an inactive worker, *can_advance_xmin is set to false.
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
@@ -1299,8 +1313,8 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
TransactionId nonremovable_xid;
/*
- * Assume the replication slot for conflict detection is created
- * before the worker starts.
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
*/
Assert(MyReplicationSlot);
@@ -1321,21 +1335,23 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
else
{
/*
- * Create a replication slot to retain information (e.g., dead tuples,
- * commit timestamps, and origins) useful for conflict detection if
- * any subscription requests it.
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins if requested by any subscription.
*
* The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid. It is
- * created even for a disabled subscription to ensure information is
- * available for detecting conflicts during the application of remote
- * changes that occur before the subscription is enabled.
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
*/
create_conflict_slot_if_not_exists();
/*
- * Only collect xmin when all workers for subscriptions with
- * retain_conflict_info enabled are running.
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
*/
*can_advance_xmin = false;
}
@@ -1350,19 +1366,19 @@ create_conflict_slot_if_not_exists(void)
{
TransactionId xmin_horizon;
- /* Exit early if the replication slot is already created and acquired */
+ /* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
- /* If the replication slot exists, acquire it and exit */
+ /* 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);
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1385,23 +1401,21 @@ create_conflict_slot_if_not_exists(void)
}
/*
- * Attempt to advance the xmin value of the replication slot used to retain
- * information useful for conflict detection.
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
*/
static void
advance_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
- new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
SpinLockAcquire(&MyReplicationSlot->mutex);
MyReplicationSlot->data.xmin = 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);
@@ -1428,10 +1442,6 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
static void
drop_conflict_slot_if_exists(void)
{
- /*
- * Avoid the overhead of scanning shared memory for a replication slot
- * that is known to have been dropped.
- */
if (conflict_slot_dropped)
return;
On Tue, Jun 10, 2025 at 11:55 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V35 patch set which includes the following changes:
Thank You for the patches, few comments:
1)
compute_min_nonremovable_xid:
+ /*
+ * Stop advancing xmin if an invalid non-removable transaction ID is
+ * found, otherwise update xmin.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ *can_advance_xmin = false;
IMO, we can not have invalid nonremovable_xid here. It may be a
possibility in a later patch where we do stop-conflict-detection for a
worker and resets nonremovable_xid to Invalid. But in patch003, should
this rather be an Assert as we want to ensure that worker's oldest is
set to slot's-xmin by this time.
2)
postgres=# create subscription sub1 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub1 WITH
(retain_conflict_info = true);
NOTICE: deleted rows will continue to accumulate for detecting
conflicts until the subscription is enabled
NOTICE: created replication slot "sub1" on publisher
CREATE SUBSCRIPTION
postgres=# alter subscription sub2 enable;
NOTICE: deleted rows will continue to accumulate for detecting
conflicts until the subscription is enabled
We should not have this 'NOTICE: deleted rows' in above 2 commands.
3)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
Shall we enable regress_testsub as well before we drop it to catch
unexpected notices/warnings if any?
4)
postgres=# alter subscription sub2 disable;
WARNING: commit timestamp and origin data required for detecting
conflicts won't be retained
HINT: Consider setting "track_commit_timestamp" to true.
Do we need "track_commit_timestamp" related WARNING while disabling
rci sub as well? I feel it should be there while enabling it alone.
thanks
Shveta
On Wed, Jun 11, 2025 at 5:34 PM shveta malik wrote:
On Tue, Jun 10, 2025 at 11:55 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V35 patch set which includes the following changes:
Thank You for the patches, few comments:
Thanks for the comments!
1)
compute_min_nonremovable_xid:+ /* + * Stop advancing xmin if an invalid non-removable transaction ID is + * found, otherwise update xmin. + */ + if (!TransactionIdIsValid(nonremovable_xid)) + *can_advance_xmin = false;IMO, we can not have invalid nonremovable_xid here. It may be a possibility in
a later patch where we do stop-conflict-detection for a worker and resets
nonremovable_xid to Invalid. But in patch003, should this rather be an Assert
as we want to ensure that worker's oldest is set to slot's-xmin by this time.
Agreed. I have removed the check and added an Assert.
2)
postgres=# create subscription sub1 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub1 WITH
(retain_conflict_info = true);
NOTICE: deleted rows will continue to accumulate for detecting conflicts
until the subscription is enabled
NOTICE: created replication slot "sub1" on publisher CREATE
SUBSCRIPTIONpostgres=# alter subscription sub2 enable;
NOTICE: deleted rows will continue to accumulate for detecting conflicts
until the subscription is enabledWe should not have this 'NOTICE: deleted rows' in above 2 commands.
Agreed, It was a miss. After thinking more, I have modified the patch to report
the NOTICE only when altering the retain_conflict_info for disabled
subscription as that aligns with the original intent for introducing the
NOTICE.
3) + +ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE); DROP +SUBSCRIPTION regress_testsub;Shall we enable regress_testsub as well before we drop it to catch unexpected
notices/warnings if any?
I have added the ENABLE test in the tap-test in 0005, because we do not have a
valid connection for subscriptions created in subscription.sql, So enabling the
sub here could cause network error with random socket info, making the test
unstable.
4)
postgres=# alter subscription sub2 disable;
WARNING: commit timestamp and origin data required for detecting conflicts
won't be retained
HINT: Consider setting "track_commit_timestamp" to true.Do we need "track_commit_timestamp" related WARNING while disabling rci
sub as well? I feel it should be there while enabling it alone.
Changed as suggested.
Here is the V35 patch set which includes the following changes:
0001:
Merged original V35-0001 and 0002.
0002:
* Addressed the comments above.
* The launcher will now attempt to acquire the conflict detection slot
initially, allowing us to determine its existence by checking if
MyReplicationSlot is NULL.
* Merged the top-up patch provided by Amit[1].
0003:
Rebased
0004:
Rebased
0005:
Added few tests to verify the WANRINGs and NOTCE.
0006:
Rebased
0007:
Rebased
Best Regards,
Hou zj
Attachments:
v36-0007-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v36-0007-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 307cffdb00a9e35c0f524af11d37c55e2fd24021 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 12 Jun 2025 12:55:41 +0800
Subject: [PATCH v36 7/7] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 61 +------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 85 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 79 +++++++----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 11 +--
8 files changed, 121 insertions(+), 155 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e6f1cffeda5..47efa5d86a5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f2409c076b6..a734817b580 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1075,8 +1075,7 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1084,41 +1083,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
- * Do not allow changing the retain_conflict_info option when the
- * subscription is enabled or the apply worker is active, to prevent race
- * conditions arising from the new option value being acknowledged
- * asynchronously by the launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
- *
* Note that two_phase is enabled (aka changed from 'false' to 'true') on
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
@@ -1392,30 +1356,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Note that workers may still survive even if the
- * subscription has been disabled.
- *
- * Ensure workers have already been exited to avoid the
- * race conditions as described in CheckAlterSubOption().
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
CheckSubConflictInfoRetention(opts.retainconflictinfo,
true, false);
- /*
- * CheckAlterSubOption() ensures that the subscription is
- * disabled.
- */
- Assert(!sub->enabled);
-
/*
* Provide a notice if retain_conflict_info is enabled for
* a disabled subscription, reminding the user to enable
@@ -1424,7 +1367,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* retain_conflict_info can be altered only for disabled
* subscriptions.
*/
- if (opts.retainconflictinfo)
+ if (opts.retainconflictinfo && !sub->enabled)
ereport(NOTICE,
errmsg("deleted rows will continue to accumulate for detecting conflicts until the subscription is enabled"));
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 81b26c4d0e5..24f2df40abd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,12 +105,12 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void create_conflict_slot_if_not_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -309,8 +309,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -329,13 +328,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -458,10 +454,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1182,7 +1176,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1255,8 +1249,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1325,7 +1318,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1334,13 +1327,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1360,10 +1347,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1456,14 +1447,48 @@ create_conflict_slot_if_not_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return;
+
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
/* first write new xmin to disk, so we know what's up after a crash */
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fcdbf5323dc..55e4f485b23 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,8 +309,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -483,8 +483,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4206,6 +4204,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4233,17 +4233,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4251,7 +4254,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4360,7 +4363,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4452,22 +4455,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4480,7 +4477,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4506,7 +4503,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4525,7 +4522,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4560,7 +4557,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4717,6 +4714,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 was changed. 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 ||
@@ -4726,7 +4732,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,
@@ -5406,28 +5413,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled at startup. The
- * replication slot for conflict detection may not be created yet, or
- * might soon be dropped as the launcher sees retain_conflict_info as
- * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
- * when the slot is absent or at risk of being dropped, a restart is
- * initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index ba7e5a184cb..26b71fab763 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,17 +228,10 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
+# Test the WARNING and NOTICE messages related to retain_conflict_info during
+# subscription DDLs
##################################################
-# Alter failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
--
2.30.0.windows.2
v36-0001-Retain-the-information-useful-for-detecting-conf.patchapplication/octet-stream; name=v36-0001-Retain-the-information-useful-for-detecting-conf.patchDownload
From d673cd7c9d6db292fb1fe50648f68d0764fbc38b Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v36 1/7] Retain the information useful for detecting conflicts
in logical replication
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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these conflicts.
This patch allows each apply worker to maintain the non-removable transaction ID
in the shared memory following the steps described above.
This patch allows each apply worker to maintain the non-removable transaction
ID in the shared memory following the steps described above. Additionally, the
logical replication launcher will create and maintain a replication slot named
pg_conflict_detection. The launcher will periodically collects the
oldest_nonremovable_xid from all apply workers, computes the minimum
transaction ID, advances the xmin value of the replication slot if it precedes
the computed value.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/create_subscription.sgml | 4 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/commands/subscriptioncmds.c | 2 +-
src/backend/replication/logical/launcher.c | 193 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/worker.c | 523 +++++++++++++++++-
src/backend/replication/slot.c | 36 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/logicallauncher.h | 1 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 11 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
21 files changed, 984 insertions(+), 26 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 021153b2a5f..20b74d92006 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,6 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c67688cbf5f..1fd0f908554 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 1914859b2ee..5f19981f9c3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 1c3c051403d..031ae021f5b 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void create_conflict_slot_if_not_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -441,6 +447,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1105,7 +1112,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1137,6 +1147,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after a
+ * restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1146,6 +1164,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1166,12 +1186,17 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1206,6 +1231,19 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!sublist)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1233,6 +1271,157 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Compute the minimum non-removable transaction ID from all apply workers.
+ * Store the result in *xmin.
+ *
+ * If the slot cannot be advanced during this cycle, due to either a disabled
+ * subscription or an inactive worker, *can_advance_xmin is set to false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * Assume the replication slot for conflict detection is created
+ * before the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /*
+ * Stop advancing xmin if an invalid non-removable transaction ID is
+ * found, otherwise update xmin.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ *can_advance_xmin = false;
+ else if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information (e.g., dead tuples,
+ * commit timestamps, and origins) useful for conflict detection if
+ * any subscription requests it.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid. It is
+ * created even for a disabled subscription to ensure information is
+ * available for detecting conflicts during the application of remote
+ * changes that occur before the subscription is enabled.
+ */
+ create_conflict_slot_if_not_exists();
+
+ /*
+ * Only collect xmin when all workers for subscriptions are running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information 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 (acquire_conflict_slot_if_exists())
+ 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(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
+ new_xmin));
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->data.xmin = 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;
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 67655111875..85239f6c316 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 a23262957ac..c864f7f52b0 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,409 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 600b87fa9cb..41adf71c4c8 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,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/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..7b29f1814db 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 eb0b93b1114..e03e123a2ff 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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..c9ef5259b68 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,17 @@ 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 a8346cda633..b77253e35e4 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2554,6 +2554,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v36-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v36-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 4a7c51bab652c1aac40cd9f3c7384ecfaf413eae Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 16:27:46 +0800
Subject: [PATCH v36 2/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 | 11 +
doc/src/sgml/logical-replication.sgml | 6 +
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 35 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 223 ++++++++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 110 +++++----
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 84 ++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 53 ++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 4 +
src/include/replication/worker_internal.h | 3 +-
src/test/regress/expected/subscription.out | 182 ++++++++------
src/test/regress/sql/subscription.sql | 17 ++
25 files changed, 715 insertions(+), 160 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..b8f9bf573ea 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is still useful for conflict detection
+ is retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 686dd441d02..5073d31ca8e 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e6f1cffeda5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..be90088bcd0 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,41 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, an additional replication
+ slot named <quote><literal>pg_conflict_detection</literal></quote>
+ will be created on the subscriber to prevent the conflict information
+ from being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..f2409c076b6 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_pub_conflict_info_retention(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,9 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +690,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 +746,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1051,7 +1075,8 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1059,6 +1084,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
+ * Do not allow changing the retain_conflict_info option when the
+ * subscription is enabled or the apply worker is active, to prevent race
+ * conditions arising from the new option value being acknowledged
+ * asynchronously by the launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
+ *
* Note that two_phase is enabled (aka changed from 'false' to 'true') on
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
@@ -1110,6 +1170,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 +1226,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 +1386,59 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Note that workers may still survive even if the
+ * subscription has been disabled.
+ *
+ * Ensure workers have already been exited to avoid the
+ * race conditions as described in CheckAlterSubOption().
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, false);
+
+ /*
+ * CheckAlterSubOption() ensures that the subscription is
+ * disabled.
+ */
+ Assert(!sub->enabled);
+
+ /*
+ * Provide a notice if retain_conflict_info is enabled for
+ * a disabled subscription, reminding the user to enable
+ * the subscription to prevent the accumulation of dead
+ * tuples. A warning is not issued since
+ * retain_conflict_info can be altered only for disabled
+ * subscriptions.
+ */
+ if (opts.retainconflictinfo)
+ ereport(NOTICE,
+ errmsg("deleted rows will continue to accumulate for detecting conflicts until the subscription is enabled"));
+
+ /*
+ * 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] =
@@ -1347,6 +1461,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Perform the track_commit_timestamp check only when enabling
+ * the subscription.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1476,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 +1498,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 +1704,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 +1721,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 +1737,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_pub_conflict_info_retention(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 +2340,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_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 031ae021f5b..799daa247e2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,6 +101,7 @@ static void logicalrep_launcher_attach_dshmem(void);
static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
TransactionId *xmin,
bool *can_advance_xmin);
static bool acquire_conflict_slot_if_exists(void);
@@ -154,6 +155,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);
@@ -302,7 +304,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -321,10 +324,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -447,7 +453,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1165,6 +1173,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1185,9 +1194,19 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
{
- compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
}
@@ -1195,7 +1214,12 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
- compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
if (w != NULL)
continue; /* worker is running already */
@@ -1222,7 +1246,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1232,13 +1257,14 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Maintain the xmin value of the replication slot for conflict
- * detection if needed. Otherwise, drop the slot if we're no longer
- * retaining information useful for conflict detection.
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!sublist)
+ if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1272,17 +1298,20 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Compute the minimum non-removable transaction ID from all apply workers.
- * Store the result in *xmin.
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
*
- * If the slot cannot be advanced during this cycle, due to either a disabled
- * subscription or an inactive worker, *can_advance_xmin is set to false.
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
bool *can_advance_xmin)
{
- if (!*can_advance_xmin)
+ if (!retain_conflict_info || !*can_advance_xmin)
return;
if (worker)
@@ -1290,8 +1319,8 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
TransactionId nonremovable_xid;
/*
- * Assume the replication slot for conflict detection is created
- * before the worker starts.
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
*/
Assert(MyReplicationSlot);
@@ -1299,33 +1328,32 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- /*
- * Stop advancing xmin if an invalid non-removable transaction ID is
- * found, otherwise update xmin.
- */
- if (!TransactionIdIsValid(nonremovable_xid))
- *can_advance_xmin = false;
- else if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
*xmin = nonremovable_xid;
}
else
{
/*
- * Create a replication slot to retain information (e.g., dead tuples,
- * commit timestamps, and origins) useful for conflict detection if
- * any subscription requests it.
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins if requested by any subscription.
*
* The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid. It is
- * created even for a disabled subscription to ensure information is
- * available for detecting conflicts during the application of remote
- * changes that occur before the subscription is enabled.
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
*/
create_conflict_slot_if_not_exists();
/*
- * Only collect xmin when all workers for subscriptions are running.
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
*/
*can_advance_xmin = false;
}
@@ -1356,16 +1384,16 @@ create_conflict_slot_if_not_exists(void)
{
TransactionId xmin_horizon;
- /* Exit early if the replication slot is already created and acquired */
+ /* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
- /* If the replication slot exists, acquire it and exit */
+ /* If the replication slot exists, acquire it, and exit */
if (acquire_conflict_slot_if_exists())
return;
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false,
- RS_PERSISTENT, false, false, false);
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1386,23 +1414,21 @@ 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.
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
*/
static void
advance_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
- new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
SpinLockAcquire(&MyReplicationSlot->mutex);
MyReplicationSlot->data.xmin = 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);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c864f7f52b0..f7434b0f6cd 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -147,6 +147,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -173,6 +174,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"
@@ -445,6 +447,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4117,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4137,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5256,28 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled at startup. The
+ * replication slot for conflict detection may not be created yet, or
+ * might soon be dropped as the launcher sees retain_conflict_info as
+ * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
+ * when the slot is absent or at risk of being dropped, a restart is
+ * initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5434,13 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true);
+
proc_exit(0);
}
@@ -5754,3 +5803,34 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue a warning if track_commit_timestamp is not enabled when
+ * check_commit_ts is set to true.
+ *
+ * Issue a warning if the subscription is being disabled.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_commit_ts,
+ bool disabling_sub)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (check_commit_ts && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (disabling_sub)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 37432e66efd..13d57dd3b13 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 7417eab6aef..945b3fce670 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 940fc77fc2e..3a3c532db47 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();
}
@@ -2017,9 +2017,11 @@ 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 for each subscription and to
+ * create the conflict detection slot when any subscription has the
+ * retain_conflict_info option enabled.
*/
static void
check_new_cluster_subscription_configuration(void)
@@ -2027,6 +2029,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)
@@ -2052,6 +2056,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);
@@ -2114,6 +2143,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 24e0100c9f0..c86010c2b89 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 620830feb9d..dd6fd07d2b0 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..635174467e5 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,8 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_commit_ts,
+ bool disabling_sub);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index c9ef5259b68..576626c6557 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -256,7 +256,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..4a529002ce0 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,48 @@ 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 warnings will occur because track_commit_timestamp is not enabled
+-- and the subscription is disabled
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = true);
+WARNING: commit timestamp and origin data required for detecting conflicts won't be retained
+HINT: Consider setting "track_commit_timestamp" to true.
+WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled
+HINT: Consider setting retain_conflict_info to false.
+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 007c9e70374..22b7ac128ce 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,23 @@ 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 warnings will occur because track_commit_timestamp is not enabled
+-- and the subscription is disabled
+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;
--
2.30.0.windows.2
v36-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v36-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 14d7371145b4a45fb8332117d2ced5769f968c7d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 12 Jun 2025 12:22:50 +0800
Subject: [PATCH v36 3/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 | 43 ++++++
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 | 111 ++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 326 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20b74d92006..355a4ac2977 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5395,6 +5395,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 799daa247e2..81b26c4d0e5 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;
@@ -103,10 +106,12 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -456,6 +461,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1174,6 +1181,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1206,7 +1214,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1219,7 +1228,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1257,14 +1266,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements:
+ * - Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ * - Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information.
+ * - if required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1305,17 +1319,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1326,8 +1344,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1356,6 +1388,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1448,6 +1486,31 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1491,7 +1554,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;
@@ -1568,6 +1631,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 f7434b0f6cd..30269a36f6c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -455,6 +455,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3867,7 +3869,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4143,6 +4146,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4290,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4364,6 +4378,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4422,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4445,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4467,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4557,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 41adf71c4c8..0d52242fc89 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"},
};
/*
@@ -1612,6 +1613,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();
}
@@ -1730,6 +1736,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;
}
@@ -1997,6 +2009,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 d3d28a263fa..846023894cc 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 7b29f1814db..d347d3d67f6 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 e03e123a2ff..a8d0844e9cf 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 576626c6557..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v36-0004-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v36-0004-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 3d6c0588c56322dff468a279672c32cc1b12bacb Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v36 4/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/worker.c | 64 ++++++++++++++++--------
2 files changed, 48 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 355a4ac2977..d97eef1216d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5421,7 +5421,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 30269a36f6c..1b3da3b76c4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -483,6 +483,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4146,10 +4148,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4429,6 +4427,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4436,6 +4453,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4477,9 +4495,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4508,19 +4525,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v36-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v36-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 4fb88f0022ad76106e7d5de3f9ebc71c8afe74a0 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v36 5/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 181 ++++++++++++++++++++++-
1 file changed, 179 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..b08e3cced80 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 @@ $node_publisher->start;
# 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,181 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows will continue to accumulate for detecting conflicts until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v36-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v36-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From e7e30ab1a8f601bf8f6226b14e32e52bc43d57cf Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 4 Jun 2025 16:40:43 +0800
Subject: [PATCH v36 6/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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 10 +-
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 +++-
14 files changed, 268 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index b8f9bf573ea..24a2e9114d9 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 5073d31ca8e..f75ae3f01f2 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index be90088bcd0..778b51e335c 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,10 +445,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
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
- <literal>false</literal>. If set to true, an additional replication
- slot named <quote><literal>pg_conflict_detection</literal></quote>
- will be created on the subscriber to prevent the conflict information
- from being removed.
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
</para>
<para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 1b3da3b76c4..fcdbf5323dc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2816,17 +2816,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. */
@@ -3164,18 +3175,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 1c12ddbae49..2a084d3f3f0 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 846023894cc..6b375f555a8 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 b08e3cced80..ba7e5a184cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -284,10 +289,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
On Thu, Jun 12, 2025 at 11:34 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V35 patch set which includes the following changes:
Thank You for the patches. Few comments:
1)
Since now we create slot for rci enabled subscription, it will require
wal_level >= replica even on subscribers. We shall mention this in the
docs.
2)
postgres=# alter subscription sub3 set (retain_conflict_info=true);
NOTICE: deleted rows will continue to accumulate for detecting
conflicts until the subscription is enabled
ALTER SUBSCRIPTION
postgres=# alter subscription sub3 disable;
WARNING: deleted rows to detect conflicts would not be removed until
the subscription is enabled
HINT: Consider setting retain_conflict_info to false.
I feel we shall have the same message in both the cases as we are
trying to convey the exact same thing concerning deleted rows
accumulation..
thanks
Shveta
On Thu, Jun 12, 2025 at 4:22 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Jun 12, 2025 at 11:34 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V35 patch set which includes the following changes:
Thank You for the patches. Few comments:
1)
Since now we create slot for rci enabled subscription, it will require
wal_level >= replica even on subscribers. We shall mention this in the
docs.2)
postgres=# alter subscription sub3 set (retain_conflict_info=true);
NOTICE: deleted rows will continue to accumulate for detecting
conflicts until the subscription is enabled
ALTER SUBSCRIPTIONpostgres=# alter subscription sub3 disable;
WARNING: deleted rows to detect conflicts would not be removed until
the subscription is enabled
HINT: Consider setting retain_conflict_info to false.I feel we shall have the same message in both the cases as we are
trying to convey the exact same thing concerning deleted rows
accumulation..
Few more comments:
3)
+static void
+create_conflict_slot_if_not_exists(void)
+{
- /* Exit early if the replication slot is already created and acquired */
+ /* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
- /* If the replication slot exists, acquire it and exit */
+ /* If the replication slot exists, acquire it, and exit */
if (acquire_conflict_slot_if_exists())
return;
We never release the slot explicitly in the launcher and thus do not
need 'If the replication slot exists, acquire it' code part here. For
the cases 1) when slot is dropped and recreated 2) slot is acquired by
launcher on restart; 'if (MyReplicationSlot)' check is enough.
4)
/*
* Common checks for altering failover and two_phase options.
*/
@@ -1051,7 +1075,8 @@ CheckAlterSubOption(Subscription *sub, const
char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
Please update the comment atop CheckAlterSubOption to mention
retain_conflict_info as well.
5)
In CheckAlterSubOption(), we need to ensure that when
'slot_needs_update' is true, it is either failover or two_phase but
not rci. Such Assert can be added.
thanks
Shveta
On Fri, Jun 13, 2025 at 2:52 PM shveta malik wrote:
On Thu, Jun 12, 2025 at 4:22 PM shveta malik <shveta.malik@gmail.com>
wrote:On Thu, Jun 12, 2025 at 11:34 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V35 patch set which includes the following changes:
Thank You for the patches. Few comments:
Thanks for the comments!
1)
Since now we create slot for rci enabled subscription, it will require
wal_level >= replica even on subscribers. We shall mention this in the
docs.
Added.
2)
postgres=# alter subscription sub3 set (retain_conflict_info=true);
NOTICE: deleted rows will continue to accumulate for detecting
conflicts until the subscription is enabled ALTER SUBSCRIPTIONpostgres=# alter subscription sub3 disable;
WARNING: deleted rows to detect conflicts would not be removed until
the subscription is enabled
HINT: Consider setting retain_conflict_info to false.I feel we shall have the same message in both the cases as we are
trying to convey the exact same thing concerning deleted rows
accumulation..
Changed.
Few more comments:
3)
+static void +create_conflict_slot_if_not_exists(void) +{ - /* Exit early if the replication slot is already created and acquired */ + /* Exit early, if the replication slot is already created and acquired + */ if (MyReplicationSlot) return;- /* If the replication slot exists, acquire it and exit */ + /* If the replication slot exists, acquire it, and exit */ if (acquire_conflict_slot_if_exists()) return;We never release the slot explicitly in the launcher and thus do not need 'If the
replication slot exists, acquire it' code part here. For the cases 1) when slot is
dropped and recreated 2) slot is acquired by launcher on restart; 'if
(MyReplicationSlot)' check is enough.
Agreed. I have removed this check.
4) /* * Common checks for altering failover and two_phase options. */ @@ -1051,7 +1075,8 @@ CheckAlterSubOption(Subscription *sub, const char *option, * two_phase options. */ Assert(strcmp(option, "failover") == 0 || - strcmp(option, "two_phase") == 0); + strcmp(option, "two_phase") == 0 || + strcmp(option, "retain_conflict_info") == 0);Please update the comment atop CheckAlterSubOption to mention
retain_conflict_info as well.
Added.
5)
In CheckAlterSubOption(), we need to ensure that when 'slot_needs_update'
is true, it is either failover or two_phase but not rci. Such Assert can be added.
Added an Assert().
0001:
* Removed the slot acquisition as suggested above.
0002:
* Addressed the comments above.
0003:
Rebased
0004:
Rebased
0005:
Rebased
0006:
Rebased
0007:
Rebased
Best Regards,
Hou zj
Attachments:
v37-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v37-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From 4767716bce1ffa2dab6b7d4dc5e848a24da5557c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 4 Jun 2025 16:40:43 +0800
Subject: [PATCH v37 6/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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 10 +-
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 +++-
14 files changed, 268 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index b8f9bf573ea..24a2e9114d9 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> 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.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 5073d31ca8e..f75ae3f01f2 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 0d6528716ce..ae3220b2ff6 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,10 +445,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
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
- <literal>false</literal>. If set to true, an additional replication
- slot named <quote><literal>pg_conflict_detection</literal></quote>
- will be created on the subscriber to prevent the conflict information
- from being removed.
+ <literal>false</literal>. If set to true, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and an
+ additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
</para>
<para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 322ea0c8b41..b1394f2d7cc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2816,17 +2816,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. */
@@ -3164,18 +3175,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 1c12ddbae49..2a084d3f3f0 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 846023894cc..6b375f555a8 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 2acfe736a94..711830b88c8 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -284,10 +289,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v37-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v37-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 5acfd7e5d63a18678b605bf5d94bdc0da04b4cc5 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 16 Jun 2025 10:06:14 +0800
Subject: [PATCH v37 2/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 | 11 +
doc/src/sgml/logical-replication.sgml | 6 +
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 37 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 215 ++++++++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 102 +++++----
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 86 ++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 53 ++++-
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/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/worker_internal.h | 3 +-
src/test/regress/expected/subscription.out | 182 ++++++++-------
src/test/regress/sql/subscription.sql | 17 ++
25 files changed, 707 insertions(+), 157 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..b8f9bf573ea 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is still useful for conflict detection
+ is retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 686dd441d02..5073d31ca8e 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e6f1cffeda5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..0d6528716ce 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,43 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ 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
+ <literal>false</literal>. If set to true, an additional replication
+ slot named <quote><literal>pg_conflict_detection</literal></quote>
+ will be created on the subscriber to prevent the conflict information
+ from being removed.
+ </para>
+
+ <para>
+ 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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is also a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..1650917abc7 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_pub_conflict_info_retention(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,9 @@ 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.")));
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +690,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 +746,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,7 +1064,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
@@ -1051,7 +1076,14 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
+ /*
+ * Altering the retain_conflict_info option does not update the slot on the
+ * publisher.
+ */
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1059,6 +1091,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
+ * Do not allow changing the retain_conflict_info option when the
+ * subscription is enabled or the apply worker is active, to prevent race
+ * conditions arising from the new option value being acknowledged
+ * asynchronously by the launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
+ *
* Note that two_phase is enabled (aka changed from 'false' to 'true') on
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
@@ -1110,6 +1177,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 +1233,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 +1393,41 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Note that workers may still survive even if the
+ * subscription has been disabled.
+ *
+ * Ensure workers have already been exited to avoid the
+ * race conditions as described in CheckAlterSubOption().
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1450,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Perform the track_commit_timestamp check only when enabling
+ * the subscription.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1466,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 +1488,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 +1694,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 +1711,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 +1727,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_pub_conflict_info_retention(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 +2330,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_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 531f8c88639..8898f67b338 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,6 +101,7 @@ static void logicalrep_launcher_attach_dshmem(void);
static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
TransactionId *xmin,
bool *can_advance_xmin);
static bool acquire_conflict_slot_if_exists(void);
@@ -154,6 +155,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);
@@ -302,7 +304,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -321,10 +324,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -447,7 +453,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1165,6 +1173,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1185,9 +1194,19 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
{
- compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
}
@@ -1195,7 +1214,12 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
- compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
if (w != NULL)
continue; /* worker is running already */
@@ -1222,7 +1246,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1232,13 +1257,14 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Maintain the xmin value of the replication slot for conflict
- * detection if needed. Otherwise, drop the slot if we're no longer
- * retaining information useful for conflict detection.
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!sublist)
+ if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1272,17 +1298,20 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Compute the minimum non-removable transaction ID from all apply workers.
- * Store the result in *xmin.
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
*
- * If the slot cannot be advanced during this cycle, due to either a disabled
- * subscription or an inactive worker, *can_advance_xmin is set to false.
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
bool *can_advance_xmin)
{
- if (!*can_advance_xmin)
+ if (!retain_conflict_info || !*can_advance_xmin)
return;
if (worker)
@@ -1290,8 +1319,8 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
TransactionId nonremovable_xid;
/*
- * Assume the replication slot for conflict detection is created
- * before the worker starts.
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
*/
Assert(MyReplicationSlot);
@@ -1299,33 +1328,32 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- /*
- * Stop advancing xmin if an invalid non-removable transaction ID is
- * found, otherwise update xmin.
- */
- if (!TransactionIdIsValid(nonremovable_xid))
- *can_advance_xmin = false;
- else if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
*xmin = nonremovable_xid;
}
else
{
/*
- * Create a replication slot to retain information (e.g., dead tuples,
- * commit timestamps, and origins) useful for conflict detection if
- * any subscription requests it.
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins if requested by any subscription.
*
* The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid. It is
- * created even for a disabled subscription to ensure information is
- * available for detecting conflicts during the application of remote
- * changes that occur before the subscription is enabled.
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
*/
create_conflict_slot_if_not_exists();
/*
- * Only collect xmin when all workers for subscriptions are running.
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
*/
*can_advance_xmin = false;
}
@@ -1382,23 +1410,21 @@ 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.
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
*/
static void
advance_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
- new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
SpinLockAcquire(&MyReplicationSlot->mutex);
MyReplicationSlot->data.xmin = 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);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c864f7f52b0..6f1eb6d41fb 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -147,6 +147,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -173,6 +174,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"
@@ -445,6 +447,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4117,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4137,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5256,28 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled at startup. The
+ * replication slot for conflict detection may not be created yet, or
+ * might soon be dropped as the launcher sees retain_conflict_info as
+ * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
+ * when the slot is absent or at risk of being dropped, a restart is
+ * initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5434,13 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5754,3 +5803,36 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue a warning if track_commit_timestamp is not enabled when
+ * check_commit_ts is set to true.
+ *
+ * Issue a message if the subscription is disabled, with the error level
+ * provided by elevel_for_disabled_sub.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_commit_ts,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (check_commit_ts && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 37432e66efd..13d57dd3b13 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 7417eab6aef..945b3fce670 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 940fc77fc2e..3a3c532db47 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();
}
@@ -2017,9 +2017,11 @@ 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 for each subscription and to
+ * create the conflict detection slot when any subscription has the
+ * retain_conflict_info option enabled.
*/
static void
check_new_cluster_subscription_configuration(void)
@@ -2027,6 +2029,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)
@@ -2052,6 +2056,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);
@@ -2114,6 +2143,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..dc6deed5557 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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 24e0100c9f0..c86010c2b89 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 2c0b4f28c14..6597672ff20 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..27774b883d3 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_commit_ts,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index c9ef5259b68..576626c6557 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -256,7 +256,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..4a529002ce0 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,48 @@ 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 warnings will occur because track_commit_timestamp is not enabled
+-- and the subscription is disabled
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = true);
+WARNING: commit timestamp and origin data required for detecting conflicts won't be retained
+HINT: Consider setting "track_commit_timestamp" to true.
+WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled
+HINT: Consider setting retain_conflict_info to false.
+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 007c9e70374..22b7ac128ce 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,23 @@ 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 warnings will occur because track_commit_timestamp is not enabled
+-- and the subscription is disabled
+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;
--
2.30.0.windows.2
v37-0001-Retain-the-information-useful-for-detecting-conf.patchapplication/octet-stream; name=v37-0001-Retain-the-information-useful-for-detecting-conf.patchDownload
From 541c0af32897942d4fe1d94c0beb8ea0df8176a3 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v37 1/7] Retain the information useful for detecting conflicts
in logical replication
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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these conflicts.
This patch allows each apply worker to maintain the non-removable transaction ID
in the shared memory following the steps described above.
This patch allows each apply worker to maintain the non-removable transaction
ID in the shared memory following the steps described above. Additionally, the
logical replication launcher will create and maintain a replication slot named
pg_conflict_detection. The launcher will periodically collects the
oldest_nonremovable_xid from all apply workers, computes the minimum
transaction ID, advances the xmin value of the replication slot if it precedes
the computed value.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/create_subscription.sgml | 4 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/commands/subscriptioncmds.c | 2 +-
src/backend/replication/logical/launcher.c | 189 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/worker.c | 523 +++++++++++++++++-
src/backend/replication/slot.c | 36 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/logicallauncher.h | 1 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 11 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
21 files changed, 980 insertions(+), 26 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 021153b2a5f..20b74d92006 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,6 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c67688cbf5f..1fd0f908554 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 1c3c051403d..531f8c88639 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void create_conflict_slot_if_not_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -441,6 +447,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1105,7 +1112,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1137,6 +1147,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after a
+ * restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1146,6 +1164,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1166,12 +1186,17 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1206,6 +1231,19 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!sublist)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1233,6 +1271,153 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Compute the minimum non-removable transaction ID from all apply workers.
+ * Store the result in *xmin.
+ *
+ * If the slot cannot be advanced during this cycle, due to either a disabled
+ * subscription or an inactive worker, *can_advance_xmin is set to false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * Assume the replication slot for conflict detection is created
+ * before the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /*
+ * Stop advancing xmin if an invalid non-removable transaction ID is
+ * found, otherwise update xmin.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ *can_advance_xmin = false;
+ else if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information (e.g., dead tuples,
+ * commit timestamps, and origins) useful for conflict detection if
+ * any subscription requests it.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid. It is
+ * created even for a disabled subscription to ensure information is
+ * available for detecting conflicts during the application of remote
+ * changes that occur before the subscription is enabled.
+ */
+ create_conflict_slot_if_not_exists();
+
+ /*
+ * Only collect xmin when all workers for subscriptions are running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information 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;
+
+ 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(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin,
+ new_xmin));
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->data.xmin = 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;
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 67655111875..85239f6c316 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 a23262957ac..c864f7f52b0 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,409 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c64f020742f..7bd7b02d42d 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,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/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..7b29f1814db 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 ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..c9ef5259b68 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,17 @@ 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v37-0007-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v37-0007-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From f0ed7ec610af374416f97771f256a5ca5a23492d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 16 Jun 2025 11:04:31 +0800
Subject: [PATCH v37 7/7] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 53 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 85 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 79 +++++++----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 11 +--
8 files changed, 120 insertions(+), 148 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e6f1cffeda5..47efa5d86a5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 1650917abc7..e9ea6771caf 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1076,8 +1076,7 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
+ strcmp(option, "two_phase") == 0);
/*
* Altering the retain_conflict_info option does not update the slot on the
@@ -1091,41 +1090,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
- * Do not allow changing the retain_conflict_info option when the
- * subscription is enabled or the apply worker is active, to prevent race
- * conditions arising from the new option value being acknowledged
- * asynchronously by the launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
- *
* Note that two_phase is enabled (aka changed from 'false' to 'true') on
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
@@ -1399,21 +1363,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Note that workers may still survive even if the
- * subscription has been disabled.
- *
- * Ensure workers have already been exited to avoid the
- * race conditions as described in CheckAlterSubOption().
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
CheckSubConflictInfoRetention(opts.retainconflictinfo,
true, !sub->enabled, NOTICE);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f868f0aaeff..cf7d96e309d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,12 +105,12 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void create_conflict_slot_if_not_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -309,8 +309,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -329,13 +328,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -458,10 +454,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1182,7 +1176,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1255,8 +1249,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1325,7 +1318,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1334,13 +1327,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1360,10 +1347,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1462,14 +1453,48 @@ create_conflict_slot_if_not_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
+ return;
+
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
/* first write new xmin to disk, so we know what's up after a crash */
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b1394f2d7cc..9c03e3c1d5f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,8 +309,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -483,8 +483,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4206,6 +4204,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4233,17 +4233,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4251,7 +4254,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4360,7 +4363,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4452,22 +4455,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4480,7 +4477,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4506,7 +4503,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4525,7 +4522,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4560,7 +4557,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4717,6 +4714,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 was changed. 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 ||
@@ -4726,7 +4732,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,
@@ -5406,28 +5413,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled at startup. The
- * replication slot for conflict detection may not be created yet, or
- * might soon be dropped as the launcher sees retain_conflict_info as
- * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
- * when the slot is absent or at risk of being dropped, a restart is
- * initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 711830b88c8..313dd4ba14d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,17 +228,10 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
+# Test the WARNING and NOTICE messages related to retain_conflict_info during
+# subscription DDLs
##################################################
-# Alter failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
--
2.30.0.windows.2
v37-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v37-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 861f2798c00423b4ebce7a633b6940a872e696f8 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 12 Jun 2025 12:22:50 +0800
Subject: [PATCH v37 3/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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 332 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20b74d92006..355a4ac2977 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5395,6 +5395,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 8898f67b338..fcd7f591c0e 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;
@@ -103,10 +106,12 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -456,6 +461,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1174,6 +1181,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1206,7 +1214,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1219,7 +1228,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1257,14 +1266,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements:
+ * - Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ * - Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information.
+ * - if required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1305,17 +1319,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1326,8 +1344,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1356,6 +1388,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1444,6 +1482,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1487,7 +1556,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;
@@ -1564,6 +1633,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 6f1eb6d41fb..0f048027184 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -455,6 +455,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3867,7 +3869,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4143,6 +4146,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4290,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4364,6 +4378,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4422,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4445,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4467,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4557,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 7bd7b02d42d..cb376022c4a 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"},
};
/*
@@ -1656,6 +1657,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();
}
@@ -1774,6 +1780,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;
}
@@ -2046,6 +2058,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 d3d28a263fa..846023894cc 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 7b29f1814db..d347d3d67f6 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 9c8c7b9840b..52b5d7aa815 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 576626c6557..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v37-0004-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v37-0004-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 8e9eceafc248057f8bcda3262735e6e1ab996797 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v37 4/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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 355a4ac2977..d97eef1216d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5421,7 +5421,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index fcd7f591c0e..f868f0aaeff 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1368,6 +1368,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0f048027184..322ea0c8b41 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -483,6 +483,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4146,10 +4148,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4429,6 +4427,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4436,6 +4453,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4477,9 +4495,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4508,19 +4525,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v37-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v37-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 575b93ee2924d25b76ecd5f2bc41c6f2eb9f0f35 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v37 5/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 181 ++++++++++++++++++++++-
1 file changed, 179 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..2acfe736a94 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 @@ $node_publisher->start;
# 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,181 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
On Thu, Jun 12, 2025 at 11:34 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Few comments on v36 patches:
==========================
1. In advance_conflict_slot_xmin(), we first save the slot to disk,
then update its effective xmin, and then do the required xmin
computation. Now, if we don't save the slot every time, there is a
risk that its value can go backwards after a restart. But OTOH, for
physical slots maintained by walsender for physical replication, we
also don't save the physical slot. However, still the system works,
see discussion in email: [1]/messages/by-id/28c8bf-68470780-3-51b29480@89454035.
As per my understanding, even if the conflict_slot's xmin moved back
after restart, it shouldn't cause any problem. Because it will anyway
be moved ahead in the next cycle, and there won't be any rows that
will get removed but are required for conflict detection. If this is
correct, then we don't need to save the slot in
advance_conflict_slot_xmin().
2.
+ *
+ * Issue a warning if track_commit_timestamp is not enabled when
+ * check_commit_ts is set to true.
+ *
+ * Issue a warning if the subscription is being disabled.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_commit_ts,
+ bool disabling_sub)
+{
+ if (!retain_conflict_info)
+ return;
+
+ if (check_commit_ts && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting
conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (disabling_sub)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until
the subscription is enabled"),
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
The quoted comments atop this function just say what it is apparent
from the code. It is better if the comments explain why we allow to
proceed when the above conditions are not met.
I think we can probably add a check here that this option requires
wal_level = replica as the launcher needs to create a physical slot to
retain the required info.
3. Isn't the new check for logical slots in
check_new_cluster_subscription_configuration() somewhat redundant with
the previous check done in
check_new_cluster_logical_replication_slots()? Can't we combine both?
Apart from this, I have made a number of changes in the comments and a
few other cosmetic changes in the attached.
[1]: /messages/by-id/28c8bf-68470780-3-51b29480@89454035
--
With Regards,
Amit Kapila.
Attachments:
v36_amit_1.patch.txttext/plain; charset=US-ASCII; name=v36_amit_1.patch.txtDownload
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index b8f9bf573ea..16310cf474f 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8088,8 +8088,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para>
<para>
If true, the information (e.g., dead tuples, commit timestamps, and
- origins) on the subscriber that is still useful for conflict detection
- is retained.
+ origins) on the subscriber that is useful for conflict detection is\
+ retained.
</para></entry>
</row>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index be90088bcd0..0e49bf09eca 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -443,18 +443,18 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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
- <literal>false</literal>. If set to true, an additional replication
- slot named <quote><literal>pg_conflict_detection</literal></quote>
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to true, a replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>
will be created on the subscriber to prevent the conflict information
from being removed.
</para>
<para>
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 <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
And even if multiple subscriptions on one node enable this option,
only one replication slot will be created.
</para>
@@ -468,7 +468,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
<para>
- This option cannot be enabled if the publisher is also a physical standby.
+ This option cannot be enabled if the publisher is a physical standby.
</para>
</listitem>
</varlistentry>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 9938fbe3e57..396cfcce1f6 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -72,7 +72,7 @@
#define SUBOPT_PASSWORD_REQUIRED 0x00000800
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
-#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
+#define SUBOPT_RETAIN_CONFLICT_INFO 0x00004000
#define SUBOPT_LSN 0x00008000
#define SUBOPT_ORIGIN 0x00010000
@@ -625,6 +625,7 @@ 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.")));
+ /* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
!opts.enabled);
@@ -1084,10 +1085,15 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* publisher cannot be modified if the slot is currently acquired by the
* existing walsender.
*
- * Do not allow changing the retain_conflict_info option when the
- * subscription is enabled or the apply worker is active, to prevent race
- * conditions arising from the new option value being acknowledged
- * asynchronously by the launcher and apply workers.
+ * Note that two_phase is enabled (aka changed from 'false' to 'true') on
+ * the publisher by the existing walsender, so we could have allowed that
+ * even when the subscription is enabled. But we kept this restriction for
+ * the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
*
* Without the restriction, a race condition may arise when a user
* disables and immediately re-enables the retain_conflict_info option. In
@@ -1118,11 +1124,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* to avoid the race conditions described above, but we maintain the
* restriction for both enable and disable operations for the sake of
* consistency.
- *
- * Note that two_phase is enabled (aka changed from 'false' to 'true') on
- * the publisher by the existing walsender, so we could have allowed that
- * even when the subscription is enabled. But we kept this restriction for
- * the sake of consistency and simplicity.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1399,7 +1400,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription has been disabled.
*
* Ensure workers have already been exited to avoid the
- * race conditions as described in CheckAlterSubOption().
+ * race conditions described in CheckAlterSubOption().
*/
if (logicalrep_workers_find(subid, true, true))
ereport(ERROR,
@@ -1417,12 +1418,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
Assert(!sub->enabled);
/*
- * Provide a notice if retain_conflict_info is enabled for
- * a disabled subscription, reminding the user to enable
- * the subscription to prevent the accumulation of dead
- * tuples. A warning is not issued since
- * retain_conflict_info can be altered only for disabled
- * subscriptions.
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
*/
if (opts.retainconflictinfo)
ereport(NOTICE,
@@ -1461,10 +1458,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
- /*
- * Perform the track_commit_timestamp check only when enabling
- * the subscription.
- */
+ /* Check track_commit_timestamp only when enabling the subscription. */
CheckSubConflictInfoRetention(sub->retainconflictinfo,
opts.enabled, !opts.enabled);
@@ -2341,14 +2335,15 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
}
/*
- * Check if the publisher's status permits enabling retain_conflict_info.
+ * Determine whether the retain_conflict_info can be enable based on the
+ * publisher's status.
*
- * 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
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
* server).
*
- * Refer to the comments atop maybe_advance_nonremovable_xid() for detailed
- * reasons.
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
*/
static void
check_pub_conflict_info_retention(WalReceiverConn *wrconn, bool retain_conflict_info)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 799daa247e2..aa763fc5ef2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1339,7 +1339,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
- * origins if requested by any subscription.
+ * origins.
*
* The slot is created before starting the apply worker to prevent it
* from unnecessarily maintaining its oldest_nonremovable_xid.
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f7434b0f6cd..10e1cb7ea45 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -5257,18 +5257,20 @@ InitializeLogRepWorker(void)
}
/*
- * Restart the worker if retain_conflict_info was enabled at startup. The
- * replication slot for conflict detection may not be created yet, or
- * might soon be dropped as the launcher sees retain_conflict_info as
- * disabled. To prevent unnecessary maintenance of oldest_nonremovable_xid
- * when the slot is absent or at risk of being dropped, a restart is
- * initiated.
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
*
* The oldest_nonremovable_xid should be initialized only when the
* retain_conflict_info is enabled before launching the worker. See
* logicalrep_worker_launch.
*/
- if (am_leader_apply_worker() && MySubscription->retainconflictinfo &&
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
On Mon, Jun 16, 2025 at 9:29 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
0001:
* Removed the slot acquisition as suggested above.0002:
* Addressed the comments above.
Thanks for the patches.
In advance_conflict_slot_xmin(), if new_xmin is same as slot's current
xmin, then shall we simply return without doing any slot- update?
Below assert indicates that new_xmin can be same as slot's current
xmin:
Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
thanks
Shveta
On Mon, Jun 16, 2025 at 7:37 PM Amit Kapila wrote:
On Thu, Jun 12, 2025 at 11:34 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Few comments on v36 patches:
==========================
1. In advance_conflict_slot_xmin(), we first save the slot to disk,
then update its effective xmin, and then do the required xmin
computation. Now, if we don't save the slot every time, there is a
risk that its value can go backwards after a restart. But OTOH, for
physical slots maintained by walsender for physical replication, we
also don't save the physical slot. However, still the system works, see discussion in email: [1].As per my understanding, even if the conflict_slot's xmin moved back
after restart, it shouldn't cause any problem. Because it will anyway
be moved ahead in the next cycle, and there won't be any rows that
will get removed but are required for conflict detection. If this is
correct, then we don't need to save the slot in advance_conflict_slot_xmin().
I think you are right, it should be OK to avoid saving the slot each time.
I have changed the patch accordingly.
2. + * + * Issue a warning if track_commit_timestamp is not enabled when + * check_commit_ts is set to true. + * + * Issue a warning if the subscription is being disabled. + */ +void +CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_commit_ts, + bool disabling_sub) +{ + if (!retain_conflict_info) + return; + + if (check_commit_ts && !track_commit_timestamp) ereport(WARNING, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"), + errhint("Consider setting \"%s\" to true.", + "track_commit_timestamp")); + + if (disabling_sub) + ereport(WARNING, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"), + errhint("Consider setting %s to false.", "retain_conflict_info"));The quoted comments atop this function just say what it is apparent
from the code. It is better if the comments explain why we allow to
proceed when the above conditions are not met.
Added.
I think we can probably add a check here that this option requires
wal_level = replica as the launcher needs to create a physical slot to
retain the required info.
Added.
After adding this ERROR, I removed the create sub (retain_conflict_info=true)
test in the subscription.sql. This is because it could cause the regression
test in 002_pg_upgrade.pl to fail as the wal_level is set to minimal in that
tap-test. I didn't add new tests since we already have tests for sub creation
in other tap tests(004_subscription.pl, 035_conflicts.pl).
Note that, the 004_subscription.pl would fail due to a crash related to commit
ca307d5. And the same has also been noticed by BF, reported in [1]/messages/by-id/CALDaNm3s-jpQTe1MshsvQ8GO=TLj233JCdkQ7uZ6pwqRVpxAdw@mail.gmail.com. Please
ignore this error temporarily.
3. Isn't the new check for logical slots in
check_new_cluster_subscription_configuration() somewhat redundant with
the previous check done in check_new_cluster_logical_replication_slots()?
Can't we combine both?
Merged as suggested.
Apart from this, I have made a number of changes in the comments and a
few other cosmetic changes in the attached.
Thanks, merged.
Here is the V38 patch set which includes the following changes:
0001:
* Addressed the comments above.
* Addressed Shveta's comments[2]/messages/by-id/CAJpy0uDJ3ofFk4FFWCf6hLQZaPb=ry45906pqcQZ+g-p3C=_JA@mail.gmail.com.
0002:
* Addressed the comments above.
* Added the document to explain that the commit timestamp data
would not be preserved during upgrade.
0003:
Rebased
0004:
Rebased
0005:
Rebased
0006:
Rebased
0007:
Rebased
[1]: /messages/by-id/CALDaNm3s-jpQTe1MshsvQ8GO=TLj233JCdkQ7uZ6pwqRVpxAdw@mail.gmail.com
[2]: /messages/by-id/CAJpy0uDJ3ofFk4FFWCf6hLQZaPb=ry45906pqcQZ+g-p3C=_JA@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v38-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v38-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From 902906ae5fe98e7b4fb4079d93880ba4721deb36 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 17 Jun 2025 13:28:18 +0800
Subject: [PATCH v38 2/2] 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 | 11 +
doc/src/sgml/logical-replication.sgml | 15 ++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 37 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 221 ++++++++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 98 +++++---
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 105 ++++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 47 +++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 59 +++++
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/worker_internal.h | 3 +-
src/test/regress/expected/subscription.out | 168 +++++++------
src/test/regress/sql/subscription.sql | 11 +
25 files changed, 721 insertions(+), 158 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 686dd441d02..fcda28d648e 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,15 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <para>
+ Note that commit timestamps and origin data retained by enabling the
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option will not be preserved during the upgrade. As a
+ result, the upgraded subscriber might be unable to detect conflicts or log
+ relevant commit timestamps and origins when applying changes from the
+ publisher occurring during the upgrade.
+ </para>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..e6f1cffeda5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the additional replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..d99b3c16d14 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,43 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>
+ will be created on the subscriber to prevent the conflict information
+ from being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..3f74600b2f1 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_pub_conflict_info_retention(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,10 @@ 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.")));
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
/*
* If built with appropriate switch, whine when regression-testing
* conventions for subscription names are violated.
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,7 +1065,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
@@ -1051,7 +1077,14 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
+ /*
+ * Altering the retain_conflict_info option does not update the slot on the
+ * publisher.
+ */
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1096,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1178,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 +1234,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 +1394,45 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Note that workers may still survive even if the
+ * subscription has been disabled.
+ *
+ * Ensure workers have already been exited to avoid the
+ * race conditions described in CheckAlterSubOption().
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1455,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1471,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 +1493,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 +1699,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 +1716,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 +1732,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_pub_conflict_info_retention(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 +2335,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enable based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8095bfc87e4..83935de9141 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,6 +101,7 @@ static void logicalrep_launcher_attach_dshmem(void);
static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
TransactionId *xmin,
bool *can_advance_xmin);
static bool acquire_conflict_slot_if_exists(void);
@@ -154,6 +155,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);
@@ -302,7 +304,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -321,10 +324,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -447,7 +453,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1165,6 +1173,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1185,9 +1194,19 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
{
- compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
}
@@ -1195,7 +1214,12 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
- compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
if (w != NULL)
continue; /* worker is running already */
@@ -1222,7 +1246,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1232,13 +1257,14 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Maintain the xmin value of the replication slot for conflict
- * detection if needed. Otherwise, drop the slot if we're no longer
- * retaining information useful for conflict detection.
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!sublist)
+ if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1272,17 +1298,20 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Compute the minimum non-removable transaction ID from all apply workers.
- * Store the result in *xmin.
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
*
- * If the slot cannot be advanced during this cycle, due to either a disabled
- * subscription or an inactive worker, *can_advance_xmin is set to false.
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
bool *can_advance_xmin)
{
- if (!*can_advance_xmin)
+ if (!retain_conflict_info || !*can_advance_xmin)
return;
if (worker)
@@ -1290,8 +1319,8 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
TransactionId nonremovable_xid;
/*
- * Assume the replication slot for conflict detection is created
- * before the worker starts.
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
*/
Assert(MyReplicationSlot);
@@ -1299,33 +1328,32 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- /*
- * Stop advancing xmin if an invalid non-removable transaction ID is
- * found, otherwise update xmin.
- */
- if (!TransactionIdIsValid(nonremovable_xid))
- *can_advance_xmin = false;
- else if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
*xmin = nonremovable_xid;
}
else
{
/*
- * Create a replication slot to retain information (e.g., dead tuples,
- * commit timestamps, and origins) useful for conflict detection if
- * any subscription requests it.
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
*
* The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid. It is
- * created even for a disabled subscription to ensure information is
- * available for detecting conflicts during the application of remote
- * changes that occur before the subscription is enabled.
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
*/
create_conflict_slot_if_not_exists();
/*
- * Only collect xmin when all workers for subscriptions are running.
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
*/
*can_advance_xmin = false;
}
@@ -1382,8 +1410,8 @@ 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.
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
*/
static void
advance_conflict_slot_xmin(TransactionId new_xmin)
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c864f7f52b0..a6fc45cb9f0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -147,6 +147,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -173,6 +174,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"
@@ -445,6 +447,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4117,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4137,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5256,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5436,13 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5754,3 +5805,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("Enabling retain_conflict_info requires \"wal_level\" >= \"replica\""),
+ errdetail("A replication slot must be created to retain conflict information."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 37432e66efd..13d57dd3b13 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 7417eab6aef..945b3fce670 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 940fc77fc2e..41fcab78ff7 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,13 +1943,13 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
* Verify that there are no logical replication slots on the new cluster and
* that the parameter settings necessary for creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
@@ -1964,8 +1964,11 @@ check_new_cluster_logical_replication_slots(void)
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
@@ -1997,12 +2000,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2129,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..4978c9d5636 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -82,6 +82,65 @@ command_checks_all(
# Reset max_active_replication_origins
$new_sub->append_conf('postgresql.conf', "max_active_replication_origins = 10");
+# Increase wal_level to allow enabling retain_conflict_info in next test
+$old_sub->append_conf('postgresql.conf', "wal_level = replica");
+
+# 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 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',
+ qq{max_replication_slots = 0
+ wal_level = replica});
+
+# 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 and wal_level
+$new_sub->append_conf(
+ 'postgresql.conf',
+ qq{max_replication_slots = 10
+ wal_level = minimal});
+
+$old_sub->append_conf('postgresql.conf', "wal_level = minimal");
+
# Cleanup
$publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$old_sub->start;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 24e0100c9f0..c86010c2b89 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 2c0b4f28c14..6597672ff20 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index c9ef5259b68..576626c6557 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -256,7 +256,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
--
2.30.0.windows.2
v38-0006-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v38-0006-Support-the-conflict-detection-for-update_delete.patchDownload
From 1eb35146eeb7cbe4afc5af645fe92126d3b947f9 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 17 Jun 2025 18:17:10 +0800
Subject: [PATCH v38 6/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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 +++-
14 files changed, 265 insertions(+), 30 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 16310cf474f..c03945406f0 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index fcda28d648e..1dfcabbf0ab 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index d99b3c16d14..f9c89cec06b 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a replication slot named
- <quote><literal>pg_conflict_detection</literal></quote>
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
will be created on the subscriber to prevent the conflict information
from being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 0b3ecab0f12..f724d20dfc3 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2816,17 +2816,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. */
@@ -3164,18 +3175,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 1c12ddbae49..2a084d3f3f0 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 846023894cc..6b375f555a8 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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 79ed5233edb..d0428ac47fb 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 2acfe736a94..711830b88c8 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -284,10 +289,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v38-0001-Retain-the-information-useful-for-detecting-conf.patchapplication/octet-stream; name=v38-0001-Retain-the-information-useful-for-detecting-conf.patchDownload
From 7dd2f7135042d66b24607bb1da705532f542ecf1 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v38 1/2] Retain the information useful for detecting conflicts
in logical replication
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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these conflicts.
This patch allows each apply worker to maintain the non-removable transaction ID
in the shared memory following the steps described above.
This patch allows each apply worker to maintain the non-removable transaction
ID in the shared memory following the steps described above. Additionally, the
logical replication launcher will create and maintain a replication slot named
pg_conflict_detection. The launcher will periodically collects the
oldest_nonremovable_xid from all apply workers, computes the minimum
transaction ID, advances the xmin value of the replication slot if it precedes
the computed value.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/create_subscription.sgml | 4 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/commands/subscriptioncmds.c | 2 +-
src/backend/replication/logical/launcher.c | 189 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/worker.c | 523 +++++++++++++++++-
src/backend/replication/slot.c | 36 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/logicallauncher.h | 1 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 11 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
21 files changed, 980 insertions(+), 26 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 021153b2a5f..20b74d92006 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4961,6 +4961,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index c67688cbf5f..1fd0f908554 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index c4d3853cbf2..cbd36641161 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 1c3c051403d..8095bfc87e4 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void create_conflict_slot_if_not_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -441,6 +447,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1105,7 +1112,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1137,6 +1147,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after a
+ * restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1146,6 +1164,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1166,12 +1186,17 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1206,6 +1231,19 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!sublist)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1233,6 +1271,153 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Compute the minimum non-removable transaction ID from all apply workers.
+ * Store the result in *xmin.
+ *
+ * If the slot cannot be advanced during this cycle, due to either a disabled
+ * subscription or an inactive worker, *can_advance_xmin is set to false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * Assume the replication slot for conflict detection is created
+ * before the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /*
+ * Stop advancing xmin if an invalid non-removable transaction ID is
+ * found, otherwise update xmin.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ *can_advance_xmin = false;
+ else if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information (e.g., dead tuples,
+ * commit timestamps, and origins) useful for conflict detection if
+ * any subscription requests it.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid. It is
+ * created even for a disabled subscription to ensure information is
+ * available for detecting conflicts during the application of remote
+ * changes that occur before the subscription is enabled.
+ */
+ create_conflict_slot_if_not_exists();
+
+ /*
+ * Only collect xmin when all workers for subscriptions are running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information 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;
+
+ 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(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on the
+ * publisher that require the data preceding the slot's xmin should have
+ * already been applied and flushed on the subscriber before the xmin is
+ * advanced. So, even if the slot's xmin regresses after a restart, it will
+ * be advanced again in the next cycle. Therefore, no data required for
+ * conflict detection will be prematurely removed.
+ */
+ return;
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 67655111875..85239f6c316 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 a23262957ac..c864f7f52b0 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,409 @@ 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. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_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;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c64f020742f..7bd7b02d42d 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,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/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..7b29f1814db 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 ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..c9ef5259b68 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,17 @@ 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v38-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v38-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From e1d15bc8f78428012dee53a6c311387396c1f33f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 12 Jun 2025 12:22:50 +0800
Subject: [PATCH v38 3/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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 332 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20b74d92006..355a4ac2977 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5395,6 +5395,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 83935de9141..2feac16e03f 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;
@@ -103,10 +106,12 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -456,6 +461,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1174,6 +1181,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1206,7 +1214,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1219,7 +1228,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1257,14 +1266,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements:
+ * - Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ * - Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information.
+ * - if required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1305,17 +1319,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1326,8 +1344,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1356,6 +1388,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1446,6 +1484,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1489,7 +1558,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;
@@ -1566,6 +1635,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 51956add970..7478c703422 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -455,6 +455,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3867,7 +3869,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4143,6 +4146,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4290,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4364,6 +4378,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4422,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4445,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4467,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4557,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 7bd7b02d42d..cb376022c4a 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"},
};
/*
@@ -1656,6 +1657,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();
}
@@ -1774,6 +1780,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;
}
@@ -2046,6 +2058,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 d3d28a263fa..846023894cc 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 7b29f1814db..d347d3d67f6 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 9c8c7b9840b..52b5d7aa815 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 576626c6557..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v38-0004-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v38-0004-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 78f25fae203a3addb0222c46576fad14bacf230d Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v38 4/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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 355a4ac2977..d97eef1216d 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5421,7 +5421,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 2feac16e03f..5a83027dfa4 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1368,6 +1368,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 7478c703422..0b3ecab0f12 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -483,6 +483,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4146,10 +4148,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4429,6 +4427,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4436,6 +4453,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4477,9 +4495,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4508,19 +4525,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v38-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v38-0005-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From c95c8c2a648241ae5b159ef959e352ec45a1a1ac Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v38 5/7] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 181 ++++++++++++++++++++++-
1 file changed, 179 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..2acfe736a94 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 @@ $node_publisher->start;
# 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,181 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v38-0007-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v38-0007-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From bd9fe3366b0436e297d511922e3eb8fb79ad3c4b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 17 Jun 2025 18:22:20 +0800
Subject: [PATCH v38 7/7] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 59 +------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 81 +++++++----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 13 +--
8 files changed, 119 insertions(+), 160 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index e6f1cffeda5..47efa5d86a5 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 3f74600b2f1..92b2c8b1d5e 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1077,14 +1077,7 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* two_phase options.
*/
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on the
- * publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1096,41 +1089,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option when
- * the subscription is enabled to prevent race conditions arising from the
- * new option value being acknowledged asynchronously by the launcher and
- * apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1400,21 +1358,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Note that workers may still survive even if the
- * subscription has been disabled.
- *
- * Ensure workers have already been exited to avoid the
- * race conditions described in CheckAlterSubOption().
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 5a83027dfa4..4dfcc1308b0 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,12 +105,12 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void create_conflict_slot_if_not_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -309,8 +309,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -329,13 +328,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -458,10 +454,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1182,7 +1176,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1255,8 +1249,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1325,7 +1318,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1334,13 +1327,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1360,10 +1347,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1462,19 +1453,48 @@ create_conflict_slot_if_not_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f724d20dfc3..e3b3e002d2c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,8 +309,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -483,8 +483,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4206,6 +4204,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4233,17 +4233,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4251,7 +4254,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4360,7 +4363,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4452,22 +4455,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4480,7 +4477,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4506,7 +4503,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4525,7 +4522,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4560,7 +4557,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4717,6 +4714,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 was changed. 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 ||
@@ -4726,7 +4732,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,
@@ -5406,30 +5413,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 711830b88c8..9156b3ce276 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,19 +228,12 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
+# Test the WARNING and NOTICE messages related to retain_conflict_info during
+# subscription DDLs
##################################################
-# Alter failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
ok( $stderr =~
/WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
--
2.30.0.windows.2
On Tue, Jun 17, 2025 at 4:26 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Mon, Jun 16, 2025 at 7:37 PM Amit Kapila wrote:
3. Isn't the new check for logical slots in
check_new_cluster_subscription_configuration() somewhat redundant with
the previous check done in check_new_cluster_logical_replication_slots()?
Can't we combine both?Merged as suggested.
Okay, it is better to update the comments atop
check_new_cluster_replication_slots() to reflect the new functionality
as well.
After the upgrade, there will be a window where the launcher could
take the time to create the conflict_slot if there exists any
subscription that has retain_conflict_info enabled, and during that
window, the update_delete conflicts won't be detected reliably. To
close that, we can probably create the conflict_slot during upgrade,
if required.
Now, because we don't copy commit_ts during upgrade, we still won't be
able to detect origin_differ conflicts reliably after upgrade. This
can happen in cases where we want to detect conflicts for the
transactions that were pending to replicate before the upgrade. Note,
we don't ensure that the subscriber receives and apply all the
transactions before the upgrade.
Similarly, no slot information after the upgrade helps to protect the
rows from being vacuumed after the upgrade of the subscriber, so
update_delete conflict also may not be detected reliably for the
transactions pending before the upgrade. I think we need to mention
this in the docs so that users can try to ensure that all pending
transactions have been applied before upgrading the subscriber, if
they want to detect all possible conflicts reliably.
--
With Regards,
Amit Kapila.
Here is the V38 patch set which includes the following changes:
Thank You for the patches. Few comments:
1)
+ <para>
+ Note that commit timestamps and origin data retained by enabling the
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option will not be preserved during the upgrade. As a
+ result, the upgraded subscriber might be unable to detect conflicts or log
+ relevant commit timestamps and origins when applying changes from the
+ publisher occurring during the upgrade.
+ </para>
This statement is true even for changes pending from 'before' the
upgrade. So we shall change last line where we mention 'during the
upgrade'
2)
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
I think this can be put in WARNING or CAUTION tags as this is
something which if neglected can result in system bloat.
3)
postgres=# create subscription sub3 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH (failover
= true, retain_conflict_info = true);
WARNING: commit timestamp and origin data required for detecting
conflicts won't be retained
HINT: Consider setting "track_commit_timestamp" to true.
ERROR: subscription "sub3" already exists
In CreateSubscription(), we shall move CheckSubConflictInfoRetention()
after sub-duplicity check. Above WARNING with the existing-sub ERROR
looks odd.
4)
In check_new_cluster_replication_slots(), earlier we were not doing
any checks for 'count of logical slots on new cluster' if there were
no logical slots on old cluster (i.e. nslots_on_old is 0). Now we are
doing a 'nslots_on_new' related check even when 'nslots_on_old' is 0
for the case when RCI is enabled. Shouldn't we skip 'nslots_on_new'
check when 'nslots_on_old' is 0?
5)
We refer to 'update_deleted' in patch1's comment when the conflict is
not yet created. Is it okay?
thanks
Shveta
On Thu, Jun 19, 2025 at 4:34 PM shveta malik <shveta.malik@gmail.com> wrote:
Here is the V38 patch set which includes the following changes:
Thank You for the patches. Few comments:
1) + <para> + Note that commit timestamps and origin data retained by enabling the + <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link> + option will not be preserved during the upgrade. As a + result, the upgraded subscriber might be unable to detect conflicts or log + relevant commit timestamps and origins when applying changes from the + publisher occurring during the upgrade. + </para>This statement is true even for changes pending from 'before' the
upgrade. So we shall change last line where we mention 'during the
upgrade'2) + <para> + Note that the information for conflict detection cannot be purged if + the subscription is disabled; thus, the information will accumulate + until the subscription is enabled. To prevent excessive accumulation, + it is recommended to disable <literal>retain_conflict_info</literal> + if the subscription will be inactive for an extended period. + </para>I think this can be put in WARNING or CAUTION tags as this is
something which if neglected can result in system bloat.3)
postgres=# create subscription sub3 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH (failover
= true, retain_conflict_info = true);
WARNING: commit timestamp and origin data required for detecting
conflicts won't be retained
HINT: Consider setting "track_commit_timestamp" to true.
ERROR: subscription "sub3" already existsIn CreateSubscription(), we shall move CheckSubConflictInfoRetention()
after sub-duplicity check. Above WARNING with the existing-sub ERROR
looks odd.4)
In check_new_cluster_replication_slots(), earlier we were not doing
any checks for 'count of logical slots on new cluster' if there were
no logical slots on old cluster (i.e. nslots_on_old is 0). Now we are
doing a 'nslots_on_new' related check even when 'nslots_on_old' is 0
for the case when RCI is enabled. Shouldn't we skip 'nslots_on_new'
check when 'nslots_on_old' is 0?5)
We refer to 'update_deleted' in patch1's comment when the conflict is
not yet created. Is it okay?
Please find few more comments:
6)
We can add in doc that pg_conflict_detection is a physical slot with
no wals-reserved.
7)
We shall error or give warning (whatever appropriate) in
ReplicationSlotAcquire() (similar to ReplicationSlotValidateName()),
that if it is pg_conflict_detection slot, then acquire is possible
only if the process is launcher. This will prevent:
a) manual/accidental drop of slot by user before launcher could acquire it.
b) usage of slot in primary_slot_name before launcher could acquire it.
It will also make lot-advance error more meaningful. Currently it
gives below error:
postgres=# select pg_replication_slot_advance
('pg_conflict_detection', pg_current_wal_lsn());
ERROR: replication slot "pg_conflict_detection" cannot be advanced
DETAIL: This slot has never previously reserved WAL, or it has been
invalidated.
thanks
Shveta
On Fri, Jun 20, 2025 at 3:22 PM shveta malik wrote:
On Thu, Jun 19, 2025 at 4:34 PM shveta malik <shveta.malik@gmail.com>
wrote:Here is the V38 patch set which includes the following changes:
Thank You for the patches. Few comments:
1) + <para> + Note that commit timestamps and origin data retained by enabling the + <linklinkend="sql-createsubscription-params-with-retain-conflict-info"><literal
retain_conflict_info</literal></link> + option will not be preserved during the upgrade. As a + result, the upgraded subscriber might be unable to detect conflicts orlog
+ relevant commit timestamps and origins when applying changes from
the
+ publisher occurring during the upgrade. + </para>This statement is true even for changes pending from 'before' the
upgrade. So we shall change last line where we mention 'during the
upgrade'
Changed.
2) + <para> + Note that the information for conflict detection cannot be purgedif
+ the subscription is disabled; thus, the information will
accumulate
+ until the subscription is enabled. To prevent excessive
accumulation,
+ it is recommended to disable
<literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period. + </para>I think this can be put in WARNING or CAUTION tags as this is
something which if neglected can result in system bloat.
Agreed, I chose CAUTION since the replication slot page[1]https://www.postgresql.org/docs/devel/warm-standby.html#STREAMING-REPLICATION-SLOTS
is using the same for the risk of WAL bloat.
3)
postgres=# create subscription sub3 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH (failover
= true, retain_conflict_info = true);
WARNING: commit timestamp and origin data required for detecting
conflicts won't be retained
HINT: Consider setting "track_commit_timestamp" to true.
ERROR: subscription "sub3" already existsIn CreateSubscription(), we shall move CheckSubConflictInfoRetention()
after sub-duplicity check. Above WARNING with the existing-sub ERROR
looks odd.
Changed.
4)
In check_new_cluster_replication_slots(), earlier we were not doing
any checks for 'count of logical slots on new cluster' if there were
no logical slots on old cluster (i.e. nslots_on_old is 0). Now we are
doing a 'nslots_on_new' related check even when 'nslots_on_old' is 0
for the case when RCI is enabled. Shouldn't we skip 'nslots_on_new'
check when 'nslots_on_old' is 0?
I agreed. In addition, I have also added a check to confirm the reserved
replication slot pg_conflict_detection does not exist on the new cluster
when migrating subscriptions with retain_conflict_info enabled.
5)
We refer to 'update_deleted' in patch1's comment when the conflict is
not yet created. Is it okay?
Moved to later patches where update_deleted is supported.
Please find few more comments:
6)
We can add in doc that pg_conflict_detection is a physical slot with no
wals-reserved.
I added the physical mark. Since we have explicitly mention the info we are
retaining, I didn't mention WAL in this version.
7)
We shall error or give warning (whatever appropriate) in
ReplicationSlotAcquire() (similar to ReplicationSlotValidateName()), that if it is
pg_conflict_detection slot, then acquire is possible only if the process is
launcher. This will prevent:a) manual/accidental drop of slot by user before launcher could acquire it.
b) usage of slot in primary_slot_name before launcher could acquire it.It will also make lot-advance error more meaningful. Currently it gives below
error:postgres=# select pg_replication_slot_advance ('pg_conflict_detection',
pg_current_wal_lsn());
ERROR: replication slot "pg_conflict_detection" cannot be advanced
DETAIL: This slot has never previously reserved WAL, or it has been
invalidated.
Added a new error message in slot acquisition as suggested.
Here is the V39 patch set which includes the following changes:
0001:
* Addressed the comments above.
0002:
* Addressed the comments above.
* Improved some comments and documentation additionally.
0003:
A new patch to migrate the conflict detection during upgrade. (Suggested by Amit[2]/messages/by-id/CAA4eK1+yP3+4CRtkqPEn5DgsaQXJogep4QyqTVSaqk=ouGqQEQ@mail.gmail.com)
0004:
Rebased
0005:
Rebased
0006:
Rebased
0007:
Rebased
0008:
Rebased
[1]: https://www.postgresql.org/docs/devel/warm-standby.html#STREAMING-REPLICATION-SLOTS
[2]: /messages/by-id/CAA4eK1+yP3+4CRtkqPEn5DgsaQXJogep4QyqTVSaqk=ouGqQEQ@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v39-0003-Migrate-the-conflict-detection-slot-to-the-new-n.patchapplication/octet-stream; name=v39-0003-Migrate-the-conflict-detection-slot-to-the-new-n.patchDownload
From d6da271d85f846466c31bb1632846dcd9f466fd3 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 16:16:49 +0800
Subject: [PATCH v39 3/8] Migrate the conflict detection slot to the new node
during an upgrade.
This patch enables pg_upgrade to create the replication slot that retains
information necessary for conflict detection such as dead tuples, commit
timestamps, and origins, for migrated subscriptions with retain_conflict_info
enabled.
---
src/backend/replication/logical/launcher.c | 7 ++--
src/backend/replication/slot.c | 7 ++--
src/backend/utils/adt/pg_upgrade_support.c | 19 ++++++++++
src/bin/pg_upgrade/pg_upgrade.c | 42 ++++++++++++++++++++--
src/bin/pg_upgrade/t/004_subscription.pl | 39 +++++++++-----------
src/include/catalog/pg_proc.dat | 4 +++
src/include/replication/logicallauncher.h | 2 ++
7 files changed, 88 insertions(+), 32 deletions(-)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 83935de9141..409cd7f2214 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,7 +105,6 @@ static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
TransactionId *xmin,
bool *can_advance_xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void create_conflict_slot_if_not_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
@@ -1348,7 +1347,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* conflict-related information is available when applying remote
* changes that occurred before the subscription was enabled.
*/
- create_conflict_slot_if_not_exists();
+ ApplyLauncherCreateConflictDetectionSlot();
/*
* Can't advance xmin of the slot unless all the subscriptions with
@@ -1379,8 +1378,8 @@ acquire_conflict_slot_if_exists(void)
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
*/
-static void
-create_conflict_slot_if_not_exists(void)
+void
+ApplyLauncherCreateConflictDetectionSlot(void)
{
TransactionId xmin_horizon;
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index d8dd458e93b..aa7ada418f8 100644
--- a/src/backend/replication/slot.c
+++ b/src/backend/replication/slot.c
@@ -359,10 +359,11 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
/*
- * The logical launcher might be creating an internal slot, so using a
- * reserved name is allowed in this case.
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
*/
- ReplicationSlotValidateName(name, IsLogicalLauncher(), ERROR);
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..fdb853bb344 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ ApplyLauncherCreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..f6b967bc3a9 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -204,12 +205,15 @@ main(int argc, char **argv)
* removed and slots would become unusable. There is a possibility that
* background processes might generate some WAL before we could create the
* slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * required downstream. The conflict detection slots is not affected by
+ * these concerns, but is created here for consistency.
*/
- if (count_old_cluster_logical_slots())
+ if (count_old_cluster_logical_slots() ||
+ old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
create_logical_replication_slots();
+ create_conflict_detection_slot();
stop_postmaster(false);
}
@@ -971,11 +975,15 @@ set_frozenxids(bool minmxid_only)
/*
* create_logical_replication_slots()
*
- * Similar to create_new_objects() but only restores logical replication slots.
+ * Similar to create_new_objects() but only restores logical replication slots
+ * if any.
*/
static void
create_logical_replication_slots(void)
{
+ if (!count_old_cluster_logical_slots())
+ return;
+
prep_status_progress("Restoring logical replication slots in the new cluster");
for (int dbnum = 0; dbnum < old_cluster.dbarr.ndbs; dbnum++)
@@ -1025,3 +1033,31 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ /*
+ * Quickly return if no subscriptions with retain_conflict_info enabled are
+ * migrated.
+ */
+ if (!old_cluster.sub_retain_conflict_info)
+ return;
+
+ prep_status("Creating the logical replication conflict detection slot in the new cluster");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/t/004_subscription.pl b/src/bin/pg_upgrade/t/004_subscription.pl
index 4978c9d5636..d4a9e194b68 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -82,9 +82,6 @@ command_checks_all(
# Reset max_active_replication_origins
$new_sub->append_conf('postgresql.conf', "max_active_replication_origins = 10");
-# Increase wal_level to allow enabling retain_conflict_info in next test
-$old_sub->append_conf('postgresql.conf', "wal_level = replica");
-
# Cleanup
$publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$old_sub->start;
@@ -104,10 +101,7 @@ $old_sub->safe_psql('postgres',
$old_sub->stop;
-$new_sub->append_conf(
- 'postgresql.conf',
- qq{max_replication_slots = 0
- wal_level = replica});
+$new_sub->append_conf('postgresql.conf', 'max_replication_slots = 0');
# pg_upgrade will fail because the new cluster has insufficient
# max_replication_slots.
@@ -133,13 +127,8 @@ command_checks_all(
'run of pg_upgrade where the new cluster has insufficient max_replication_slots'
);
-# Reset max_replication_slots and wal_level
-$new_sub->append_conf(
- 'postgresql.conf',
- qq{max_replication_slots = 10
- wal_level = minimal});
-
-$old_sub->append_conf('postgresql.conf', "wal_level = minimal");
+# Reset max_replication_slots
+$new_sub->append_conf('postgresql.conf', 'max_replication_slots = 10');
# Cleanup
$publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
@@ -258,7 +247,7 @@ rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# running status, failover, and retain_conflict_info option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -268,7 +257,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -350,7 +339,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, existence of the conflict
+# detection slot.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -364,10 +354,10 @@ $new_sub->start;
# in the upgraded instance. So regress_sub4 should still have subenabled and
# subfailover set to true, while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
"check that the subscription's running status and failover are preserved"
);
@@ -387,6 +377,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should exist
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index d3d28a263fa..359f03cd331 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11773,6 +11773,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 7b29f1814db..d8eb7ebed39 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -28,6 +28,8 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void ApplyLauncherCreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
--
2.30.0.windows.2
v39-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v39-0004-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 73d5e6c1e7b9a25862eeb48f9c9a9f22d3261d92 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 17:53:17 +0800
Subject: [PATCH v39 4/8] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 332 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d2b4ac7c205..c8b93d43d91 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 409cd7f2214..6e10fc7216b 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -455,6 +460,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1173,6 +1180,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1205,7 +1213,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1218,7 +1227,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1256,14 +1265,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements:
+ * - Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ * - Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information.
+ * - if required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1304,17 +1318,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1325,8 +1343,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1355,6 +1387,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1445,6 +1483,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1488,7 +1557,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;
@@ -1565,6 +1634,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 9c51d28e4f7..0dc8afe9a21 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -455,6 +455,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3867,7 +3869,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4143,6 +4146,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4290,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4364,6 +4378,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4422,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4445,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4467,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4557,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index aa7ada418f8..db8be252acc 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"},
};
/*
@@ -1663,6 +1664,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();
}
@@ -1781,6 +1787,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;
}
@@ -2053,6 +2065,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 359f03cd331..f566e23f785 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 d8eb7ebed39..7a848308c01 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v39-0005-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v39-0005-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 2edeea36b12a1fafcc90d1f21951ee5a5b70e5c6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v39 5/8] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index c8b93d43d91..954bdcadf02 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 6e10fc7216b..e29a0a87d20 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1367,6 +1367,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0dc8afe9a21..6948d46fed7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -483,6 +483,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4146,10 +4148,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4429,6 +4427,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4436,6 +4453,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4477,9 +4495,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4508,19 +4525,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v39-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v39-0006-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From e46007ddcaca2105649c9b624f4d681e68a26683 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v39 6/8] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 181 ++++++++++++++++++++++-
1 file changed, 179 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..2acfe736a94 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 @@ $node_publisher->start;
# 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,181 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v39-0007-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v39-0007-Support-the-conflict-detection-for-update_delete.patchDownload
From 72cedb719b92ae985418e8c65c0ab27c5900268c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 18:28:35 +0800
Subject: [PATCH v39 7/8] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 58 ++++++---
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 25 +++-
15 files changed, 278 insertions(+), 42 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 4217a2e7dee..54834d31777 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e5ba669e075..23304d71cae 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 6948d46fed7..3cf9137ed29 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2816,17 +2816,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. */
@@ -3164,18 +3175,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;
}
@@ -4058,10 +4080,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4100,10 +4122,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4138,7 +4160,7 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably
+ * subscription by the main apply worker to detect update_deleted conflict
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
@@ -4344,7 +4366,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily
+ * required to detect update_delete conflict. This check primarily
* addresses scenarios where the publisher's clock falls behind; if the
* publisher's clock is ahead, subsequent transactions will naturally bear
* later commit timestamps, conforming to the design outlined atop of
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 f566e23f785..6f39a713128 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 2acfe736a94..711830b88c8 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -284,10 +289,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v39-0008-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v39-0008-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From fd5c81a8d769721f9a844de003f966d612ee8264 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 18:43:49 +0800
Subject: [PATCH v39 8/8] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 63 +-------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 81 +++++++----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 13 +--
8 files changed, 120 insertions(+), 163 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 7b8f92c1a69..4c8b11612b9 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on the
- * publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option when
- * the subscription is enabled to prevent race conditions arising from the
- * new option value being acknowledged asynchronously by the launcher and
- * apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,22 +1353,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the subscription
- * has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker processes
- * have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e29a0a87d20..2039638bda2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -308,8 +308,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,13 +327,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -457,10 +453,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1181,7 +1175,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1254,8 +1248,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1324,7 +1317,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1333,13 +1326,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1359,10 +1346,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1461,19 +1452,48 @@ ApplyLauncherCreateConflictDetectionSlot(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 3cf9137ed29..dd6b2a09fe0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,8 +309,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -483,8 +483,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4206,6 +4204,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4233,17 +4233,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4251,7 +4254,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4360,7 +4363,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4452,22 +4455,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4480,7 +4477,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4506,7 +4503,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4525,7 +4522,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4560,7 +4557,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4717,6 +4714,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 was changed. 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 ||
@@ -4726,7 +4732,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,
@@ -5406,30 +5413,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 711830b88c8..9156b3ce276 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,19 +228,12 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
+# Test the WARNING and NOTICE messages related to retain_conflict_info during
+# subscription DDLs
##################################################
-# Alter failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
ok( $stderr =~
/WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
--
2.30.0.windows.2
v39-0001-Retain-the-information-useful-for-detecting-conf.patchapplication/octet-stream; name=v39-0001-Retain-the-information-useful-for-detecting-conf.patchDownload
From 762e2f210882deca567a96c22a0a8103cdd3e0bf Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v39 1/8] Retain the information useful for detecting conflicts
in logical replication
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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these conflicts.
This patch allows each apply worker to maintain the non-removable transaction ID
in the shared memory following the steps described above.
This patch allows each apply worker to maintain the non-removable transaction
ID in the shared memory following the steps described above. Additionally, the
logical replication launcher will create and maintain a replication slot named
pg_conflict_detection. The launcher will periodically collects the
oldest_nonremovable_xid from all apply workers, computes the minimum
transaction ID, advances the xmin value of the replication slot if it precedes
the computed value.
---
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/create_subscription.sgml | 4 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/commands/subscriptioncmds.c | 2 +-
src/backend/replication/logical/launcher.c | 189 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/worker.c | 523 +++++++++++++++++-
src/backend/replication/slot.c | 42 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/include/replication/logicallauncher.h | 1 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 10 +
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/tools/pgindent/typedefs.list | 2 +
21 files changed, 985 insertions(+), 26 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b265cc89c9d..d2b4ac7c205 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 8d7d9a2f3e8..5200992ad70 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29539,7 +29539,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29583,7 +29585,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29613,6 +29617,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29635,8 +29641,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 137ffc8d0b7..eb20dbcca4b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..37fd40252a3 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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 4aec73bcc6b..46d4e65da97 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 1c3c051403d..8095bfc87e4 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void create_conflict_slot_if_not_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -441,6 +447,7 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1105,7 +1112,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1137,6 +1147,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after a
+ * restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1146,6 +1164,8 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1166,12 +1186,17 @@ ApplyLauncherMain(Datum main_arg)
long elapsed;
if (!sub->enabled)
+ {
+ compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1206,6 +1231,19 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Maintain the xmin value of the replication slot for conflict
+ * detection if needed. Otherwise, drop the slot if we're no longer
+ * retaining information useful for conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!sublist)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1233,6 +1271,153 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Compute the minimum non-removable transaction ID from all apply workers.
+ * Store the result in *xmin.
+ *
+ * If the slot cannot be advanced during this cycle, due to either a disabled
+ * subscription or an inactive worker, *can_advance_xmin is set to false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * Assume the replication slot for conflict detection is created
+ * before the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /*
+ * Stop advancing xmin if an invalid non-removable transaction ID is
+ * found, otherwise update xmin.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ *can_advance_xmin = false;
+ else if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information (e.g., dead tuples,
+ * commit timestamps, and origins) useful for conflict detection if
+ * any subscription requests it.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid. It is
+ * created even for a disabled subscription to ensure information is
+ * available for detecting conflicts during the application of remote
+ * changes that occur before the subscription is enabled.
+ */
+ create_conflict_slot_if_not_exists();
+
+ /*
+ * Only collect xmin when all workers for subscriptions are running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information 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;
+
+ 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(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on the
+ * publisher that require the data preceding the slot's xmin should have
+ * already been applied and flushed on the subscriber before the xmin is
+ * advanced. So, even if the slot's xmin regresses after a restart, it will
+ * be advanced again in the next cycle. Therefore, no data required for
+ * conflict detection will be prematurely removed.
+ */
+ return;
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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 a23262957ac..e3deb87bbb7 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,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +396,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +443,18 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3660,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3739,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3767,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3784,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_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 */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3860,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3931,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3968,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 +4045,409 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably
+ * even for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c64f020742f..d8dd458e93b 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,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)
{
@@ -564,6 +596,12 @@ ReplicationSlotAcquire(const char *name, bool nowait, bool error_if_invalid)
Assert(name != NULL);
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acuiqred by logical replication launcher."));
+
retry:
Assert(MyReplicationSlot == NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..7b29f1814db 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 ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..5da49e1e28b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId 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 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
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 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v39-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchapplication/octet-stream; name=v39-0002-Add-a-retain_conflict_info-option-to-subscriptio.patchDownload
From a300319124e0d650868e73ff240792e363054428 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Tue, 17 Jun 2025 13:28:18 +0800
Subject: [PATCH v39 2/8] 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 | 11 +
doc/src/sgml/logical-replication.sgml | 27 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 39 +++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 225 ++++++++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 98 +++++---
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 105 +++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 91 +++++--
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 59 +++++
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/worker_internal.h | 3 +-
src/test/regress/expected/subscription.out | 168 +++++++------
src/test/regress/sql/subscription.sql | 11 +
25 files changed, 771 insertions(+), 170 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..4217a2e7dee 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,17 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <para>
+ Note that commit timestamps and origin data are not preserved during the
+ upgrade. Consequently, even with
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ enabled, the upgraded subscriber might be unable to detect conflicts or log
+ relevant commit timestamps and origins when applying changes from the
+ publisher occurring before or during the upgrade. To prevent this issue, the
+ user must ensure that all potentially conflicting changes are fully
+ replicated to the subscriber before proceeding with the upgrade.
+ </para>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2548,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 37fd40252a3..e5ba669e075 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -437,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 46d4e65da97..7b8f92c1a69 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_pub_conflict_info_retention(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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on the
+ * publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the subscription
+ * has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker processes
+ * have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 8095bfc87e4..83935de9141 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,6 +101,7 @@ static void logicalrep_launcher_attach_dshmem(void);
static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
TransactionId *xmin,
bool *can_advance_xmin);
static bool acquire_conflict_slot_if_exists(void);
@@ -154,6 +155,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);
@@ -302,7 +304,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -321,10 +324,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -447,7 +453,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = InvalidTransactionId;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1165,6 +1173,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1185,9 +1194,19 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
{
- compute_min_nonremovable_xid(NULL, &xmin, &can_advance_xmin);
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
}
@@ -1195,7 +1214,12 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
- compute_min_nonremovable_xid(w, &xmin, &can_advance_xmin);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
if (w != NULL)
continue; /* worker is running already */
@@ -1222,7 +1246,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1232,13 +1257,14 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Maintain the xmin value of the replication slot for conflict
- * detection if needed. Otherwise, drop the slot if we're no longer
- * retaining information useful for conflict detection.
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!sublist)
+ if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1272,17 +1298,20 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Compute the minimum non-removable transaction ID from all apply workers.
- * Store the result in *xmin.
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
*
- * If the slot cannot be advanced during this cycle, due to either a disabled
- * subscription or an inactive worker, *can_advance_xmin is set to false.
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
bool *can_advance_xmin)
{
- if (!*can_advance_xmin)
+ if (!retain_conflict_info || !*can_advance_xmin)
return;
if (worker)
@@ -1290,8 +1319,8 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
TransactionId nonremovable_xid;
/*
- * Assume the replication slot for conflict detection is created
- * before the worker starts.
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
*/
Assert(MyReplicationSlot);
@@ -1299,33 +1328,32 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin,
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- /*
- * Stop advancing xmin if an invalid non-removable transaction ID is
- * found, otherwise update xmin.
- */
- if (!TransactionIdIsValid(nonremovable_xid))
- *can_advance_xmin = false;
- else if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
*xmin = nonremovable_xid;
}
else
{
/*
- * Create a replication slot to retain information (e.g., dead tuples,
- * commit timestamps, and origins) useful for conflict detection if
- * any subscription requests it.
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
*
* The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid. It is
- * created even for a disabled subscription to ensure information is
- * available for detecting conflicts during the application of remote
- * changes that occur before the subscription is enabled.
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
*/
create_conflict_slot_if_not_exists();
/*
- * Only collect xmin when all workers for subscriptions are running.
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
*/
*can_advance_xmin = false;
}
@@ -1382,8 +1410,8 @@ 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.
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
*/
static void
advance_conflict_slot_xmin(TransactionId new_xmin)
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index e3deb87bbb7..9c51d28e4f7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -147,6 +147,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -173,6 +174,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"
@@ -445,6 +447,7 @@ static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
bool status_received);
static void get_candidate_xid(RetainConflictInfoData *rci_data);
@@ -4114,6 +4117,19 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
static void
maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
@@ -4121,9 +4137,13 @@ maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
- return;
+ return false;
- process_rci_phase_transition(rci_data, status_received);
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
}
/*
@@ -5236,6 +5256,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5392,6 +5436,13 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5754,3 +5805,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("Enabling retain_conflict_info requires \"wal_level\" >= \"replica\""),
+ errdetail("A replication slot must be created to retain conflict information."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index db944ec2230..e5cfa5992f4 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 7417eab6aef..945b3fce670 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 81865cd3e48..71314180535 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,20 +1943,28 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots when migrating logical
+ * slots.
+ * - Ensure conflict detection slot does not exist when migrating subscriptions
+ * with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
/* Logical slots can be migrated since PG17. */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
@@ -1964,27 +1972,48 @@ check_new_cluster_logical_replication_slots(void)
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
+
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1997,12 +2026,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2155,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 4b7a56f5b3b..69658595e0b 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 69c965bb7d0..352c8b6f376 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 c545abf6581..4978c9d5636 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -82,6 +82,65 @@ command_checks_all(
# Reset max_active_replication_origins
$new_sub->append_conf('postgresql.conf', "max_active_replication_origins = 10");
+# Increase wal_level to allow enabling retain_conflict_info in next test
+$old_sub->append_conf('postgresql.conf', "wal_level = replica");
+
+# 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 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',
+ qq{max_replication_slots = 0
+ wal_level = replica});
+
+# 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 and wal_level
+$new_sub->append_conf(
+ 'postgresql.conf',
+ qq{max_replication_slots = 10
+ wal_level = minimal});
+
+$old_sub->append_conf('postgresql.conf', "wal_level = minimal");
+
# Cleanup
$publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$old_sub->start;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..cd7cb2115c2 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,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 2c0b4f28c14..6597672ff20 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 5da49e1e28b..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -255,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
--
2.30.0.windows.2
On Fri, Jun 20, 2025 at 4:48 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V39 patch set which includes the following changes:
1.
-static void
-create_conflict_slot_if_not_exists(void)
+void
+ApplyLauncherCreateConflictDetectionSlot(void)
I am not so sure about adding ApplyLauncher in front of this function
name. I see most others exposed from this file add such a prefix, but
this one looks odd to me as it has nothing specific to the launcher,
though we use it in launcher? How about
CreateConflictDetectionSlot(void)?
2.
static void
create_logical_replication_slots(void)
{
+ if (!count_old_cluster_logical_slots())
+ return;
+
Doing this count twice (once here and once at the caller of
create_logical_replication_slots) seems redundant.
Apart from the above, attached please find a diff patch atop 0001,
0002, 0003. I think the first three patches look in a reasonable shape
now, can we merge them (0001, 0002, 0003)?
--
With Regards,
Amit Kapila.
Attachments:
v39-amit_1.diff.txttext/plain; charset=US-ASCII; name=v39-amit_1.diff.txtDownload
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 4217a2e7dee..bfb9c2fb31c 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2507,14 +2507,18 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
</para>
<para>
- Note that commit timestamps and origin data are not preserved during the
- upgrade. Consequently, even with
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
<link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
- enabled, the upgraded subscriber might be unable to detect conflicts or log
+ is enabled, the upgraded subscriber may be unable to detect conflicts or log
relevant commit timestamps and origins when applying changes from the
- publisher occurring before or during the upgrade. To prevent this issue, the
- user must ensure that all potentially conflicting changes are fully
- replicated to the subscriber before proceeding with the upgrade.
+ publisher occurred before the upgrade especially if those changes were not
+ replicated. Additionally, immediately after the upgrade, the vacuum may
+ remove the deleted rows that are required for conflict detection. This can
+ affect the changes that were not replicated before the upgrade. To ensure
+ consistent conflict tracking, users should ensure that all potentially
+ conflicting changes are replicated to the subscriber before initiating the
+ upgrade.
</para>
<para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index e5ba669e075..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -463,11 +463,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<caution>
<para>
- Note that the information for conflict detection cannot be purged if
- the subscription is disabled; thus, the information will accumulate
- until the subscription is enabled. To prevent excessive accumulation,
- it is recommended to disable <literal>retain_conflict_info</literal>
- if the subscription will be inactive for an extended period.
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
</para>
</caution>
diff --git a/src/bin/pg_upgrade/check.c b/src/bin/pg_upgrade/check.c
index 71314180535..5314e81390b 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -1966,7 +1966,10 @@ check_new_cluster_replication_slots(void)
int i_nslots_on_new;
int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated sing PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
@@ -2159,7 +2162,7 @@ check_old_cluster_for_valid_slots(void)
/*
* The name "pg_conflict_detection" (defined as
* CONFLICT_DETECTION_SLOT) has been reserved for logical
- * replication conflict detection since PG18.
+ * replication conflict detection slot since PG19.
*/
if (strcmp(slot->slotname, "pg_conflict_detection") == 0)
{
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index f6b967bc3a9..1b46293abc8 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -200,13 +200,25 @@ main(int argc, char **argv)
check_ok();
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream. The conflict detection slots is not affected by
- * these concerns, but is created here for consistency.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to
+ * WALs as it only retains the dead tuples. It is created here for
+ * consistency. Note that the new conflict detection slot uses the latest
+ * transaction ID as xmin, so it cannot protect dead tuples that existed
+ * before the upgrade. Additionally, commit timestamps and origin data
+ * are not preserved during the upgrade. So, even after creating the slot,
+ * the upgraded subscriber may be unable to detect conflicts or log
+ * relevant commit timestamps and origins when applying changes from the
+ * publisher occurred before the upgrade especially if those changes were
+ * not replicated. It can only protect tuples that might be deleted after
+ * the new cluster starts.
*/
if (count_old_cluster_logical_slots() ||
old_cluster.sub_retain_conflict_info)
Here is the V39 patch set which includes the following changes:
Few trivial comments:
1)
Currently we have this error and detail:
ERROR: Enabling retain_conflict_info requires "wal_level" >= "replica"
DETAIL: A replication slot must be created to retain conflict information.
Shall we change it to something like:
msg: "wal_level is insufficient to create slot required by retain_conflict_info"
hint: "wal_level must be set to replica or logical at server start"
2)
+ <para>
+ Note that commit timestamps and origin data are not preserved during the
+ upgrade. Consequently, even with
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ enabled, the upgraded subscriber might be unable to detect conflicts or log
+ relevant commit timestamps and origins when applying changes from the
+ publisher occurring before or during the upgrade. To prevent this
issue, the
+ user must ensure that all potentially conflicting changes are fully
+ replicated to the subscriber before proceeding with the upgrade.
+ </para>
Shall we have a NOTE tag here? This page has existing NOTE and WARNING
tags for similar situations where we are advising something to users?
thanks
Shveta
On Mon, Jun 23, 2025 at 3:02 PM Amit Kapila wrote:
On Fri, Jun 20, 2025 at 4:48 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:Here is the V39 patch set which includes the following changes:
1. -static void -create_conflict_slot_if_not_exists(void) +void +ApplyLauncherCreateConflictDetectionSlot(void)I am not so sure about adding ApplyLauncher in front of this function
name. I see most others exposed from this file add such a prefix, but
this one looks odd to me as it has nothing specific to the launcher, though we use it in launcher?
How about CreateConflictDetectionSlot(void)?
Agreed and changed as suggested.
2. static void create_logical_replication_slots(void) { + if (!count_old_cluster_logical_slots()) + return; +Doing this count twice (once here and once at the caller of
create_logical_replication_slots) seems redundant.
I have changed to check the slot numbers before entering the function and
declared a flag to save the results of Count_old_xxslots() function to avoid
counting it twice.
Apart from the above, attached please find a diff patch atop 0001,
0002, 0003. I think the first three patches look in a reasonable shape
now, can we merge them (0001, 0002, 0003)?
Thanks for the diff, it looks good to me.
I have merged 0001~0003 and the diff.
Here is the V40 patch set which includes the following changes:
0001:
* Merged V39-0001 ~ 0003 into one patch.
* Addressed the comments above.
* Addressed Shveta's comments[1]/messages/by-id/CAJpy0uChWC4MLt_d8EwmKRrYKQYfVQKJrng4KVgktr5v25HP5w@mail.gmail.com.
0002:
No change
0003:
No change
0004:
No change
0005:
No change
0006:
No change
[1]: /messages/by-id/CAJpy0uChWC4MLt_d8EwmKRrYKQYfVQKJrng4KVgktr5v25HP5w@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v40-0001-Retain-the-information-useful-for-detecting-conf.patchapplication/octet-stream; name=v40-0001-Retain-the-information-useful-for-detecting-conf.patchDownload
From df9ff1069007b44f944c391bef383ab01ec77490 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Fri, 17 Jan 2025 14:08:02 +0800
Subject: [PATCH v40 1/6] Retain the information useful for detecting conflicts
in logical replication
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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these conflicts.
This patch adds a subscription option retain_conflict_info, 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.
Each apply worker for a subscription with retain_conflict_info enabled will
maintain the non-removable transaction ID in the shared memory following the
steps described above. Additionally, the logical replication launcher will
create and maintain a replication slot named pg_conflict_detection if any local
subscription has the retain_conflict_info option enabled. The launcher will
periodically collects the oldest_nonremovable_xid from all apply workers,
computes the minimum transaction ID, advances the xmin value of the replication
slot if it precedes the computed value.
Enabling retain_conflict_info is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
During the upgrade, if any subscription on the old cluster has
retain_conflict_info enabled, create a conflict detection slot to protect
tuples that may be deleted when the new cluster starts.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 33 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 220 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 624 +++++++++++++++++-
src/backend/replication/slot.c | 43 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 71 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/tools/pgindent/typedefs.list | 2 +
44 files changed, 1830 insertions(+), 185 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index b265cc89c9d..d2b4ac7c205 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index a6d79765c1a..79fba771978 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29549,7 +29549,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29593,7 +29595,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29623,6 +29627,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..ca5fb4a27c6 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,23 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or log
+ relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade especially if those changes were not
+ replicated. Additionally, immediately after the upgrade, the vacuum may
+ remove the deleted rows that are required for conflict detection. This can
+ affect the changes that were not replicated before the upgrade. To ensure
+ consistent conflict tracking, users should ensure that all potentially
+ conflicting changes are replicated to the subscriber before initiating the
+ upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2554,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 137ffc8d0b7..eb20dbcca4b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4aec73bcc6b..7b8f92c1a69 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on the
+ * publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the subscription
+ * has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker processes
+ * have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 1c3c051403d..f5adbdd8fe1 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -296,7 +303,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -315,10 +323,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -441,6 +452,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1105,7 +1119,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1137,6 +1154,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after a
+ * restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1146,6 +1171,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1165,13 +1193,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1197,7 +1245,8 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo);
}
else
{
@@ -1206,6 +1255,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1233,6 +1296,155 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ 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();
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on the
+ * publisher that require the data preceding the slot's xmin should have
+ * already been applied and flushed on the subscriber before the xmin is
+ * advanced. So, even if the slot's xmin regresses after a restart, it will
+ * be advanced again in the next cycle. Therefore, no data required for
+ * conflict detection will be prematurely removed.
+ */
+ return;
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index 8e1e8762f62..1591c1c99d4 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,7 +609,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index a23262957ac..50e03064af1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -147,6 +147,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -173,12 +174,14 @@
#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"
#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 +278,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 +398,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -379,6 +445,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_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 +3663,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3662,6 +3742,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3688,6 +3770,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3703,8 +3787,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3717,6 +3824,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3751,6 +3863,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3814,6 +3934,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3849,7 +3971,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 +4048,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably
+ * even for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4715,6 +5256,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4871,6 +5436,13 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5233,3 +5805,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c64f020742f..6962752270d 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -564,6 +597,12 @@ ReplicationSlotAcquire(const char *name, bool nowait, bool error_if_invalid)
Assert(name != NULL);
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
retry:
Assert(MyReplicationSlot == NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index db944ec2230..e5cfa5992f4 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 7417eab6aef..945b3fce670 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 81865cd3e48..f8acb15f1ef 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,48 +1943,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots when migrating logical
+ * slots.
+ * - Ensure conflict detection slot does not exist when migrating subscriptions
+ * with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1997,12 +2029,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2158,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..69658595e0b 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.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..de369654291 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,37 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to
+ * WALs as it only retains the dead tuples. It is created here for
+ * consistency. Note that the new conflict detection slot uses the latest
+ * transaction ID as xmin, so it cannot protect dead tuples that existed
+ * before the upgrade. Additionally, commit timestamps and origin data
+ * are not preserved during the upgrade. So, even after creating the slot,
+ * the upgraded subscriber may be unable to detect conflicts or log
+ * relevant commit timestamps and origins when applying changes from the
+ * publisher occurred before the upgrade especially if those changes were
+ * not replicated. It can only protect tuples that might be deleted after
+ * the new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ create_conflict_detection_slot();
stop_postmaster(false);
}
@@ -971,11 +992,15 @@ set_frozenxids(bool minmxid_only)
/*
* create_logical_replication_slots()
*
- * Similar to create_new_objects() but only restores logical replication slots.
+ * Similar to create_new_objects() but only restores logical replication slots
+ * if any.
*/
static void
create_logical_replication_slots(void)
{
+ if (!count_old_cluster_logical_slots())
+ return;
+
prep_status_progress("Restoring logical replication slots in the new cluster");
for (int dbnum = 0; dbnum < old_cluster.dbarr.ndbs; dbnum++)
@@ -1025,3 +1050,31 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ /*
+ * Quickly return if no subscriptions with retain_conflict_info enabled are
+ * migrated.
+ */
+ if (!old_cluster.sub_retain_conflict_info)
+ return;
+
+ prep_status("Creating the logical replication conflict detection slot in the new cluster");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 c545abf6581..0d820720650 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -198,8 +246,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -209,7 +258,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -268,7 +317,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -291,7 +341,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -301,15 +352,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -328,6 +380,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..cd7cb2115c2 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,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 2c0b4f28c14..6597672ff20 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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_proc.dat b/src/include/catalog/pg_proc.dat
index d3d28a263fa..359f03cd331 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11773,6 +11773,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v40-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v40-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From b39bf725a99ebeeec236ecc466ff968ca40bfa5c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 17:53:17 +0800
Subject: [PATCH v40 2/6] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 332 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index d2b4ac7c205..c8b93d43d91 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 f5adbdd8fe1..d5f03f81b7a 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -455,6 +460,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1173,6 +1180,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1205,7 +1213,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1218,7 +1227,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1256,14 +1265,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements:
+ * - Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ * - Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information.
+ * - if required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1304,17 +1318,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1325,8 +1343,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1355,6 +1387,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1445,6 +1483,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1488,7 +1557,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;
@@ -1565,6 +1634,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 50e03064af1..f97a5d31c96 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -455,6 +455,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3867,7 +3869,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4143,6 +4146,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4283,6 +4290,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4364,6 +4378,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4389,7 +4422,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4409,12 +4445,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4422,9 +4467,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4457,6 +4557,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 6962752270d..8bea71513b7 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"},
};
/*
@@ -1663,6 +1664,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();
}
@@ -1781,6 +1787,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;
}
@@ -2053,6 +2065,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 359f03cd331..f566e23f785 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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v40-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v40-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 3290dd4651a85634807170b105da65389aafe18a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v40 3/6] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index c8b93d43d91..954bdcadf02 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index d5f03f81b7a..e4426fa0153 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1367,6 +1367,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f97a5d31c96..7d26c2e85d4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -483,6 +483,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4146,10 +4148,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4429,6 +4427,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4436,6 +4453,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4477,9 +4495,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4508,19 +4525,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v40-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v40-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 809f9cc639e8ceee75b9adfb71a9aae07c8c1d8a Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v40 4/6] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 181 ++++++++++++++++++++++-
1 file changed, 179 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..2acfe736a94 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 @@ $node_publisher->start;
# 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,181 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v40-0005-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v40-0005-Support-the-conflict-detection-for-update_delete.patchDownload
From dcbb6c90dc1a92d42203d2cb909b1192a07446bf Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 18:28:35 +0800
Subject: [PATCH v40 5/6] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 58 ++++++---
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 25 +++-
15 files changed, 278 insertions(+), 42 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index ca5fb4a27c6..43ffd3236ed 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 7d26c2e85d4..6c52280a212 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2816,17 +2816,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. */
@@ -3164,18 +3175,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;
}
@@ -4058,10 +4080,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4100,10 +4122,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4138,7 +4160,7 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably
+ * subscription by the main apply worker to detect update_deleted conflict
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
@@ -4344,7 +4366,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily
+ * required to detect update_delete conflict. This check primarily
* addresses scenarios where the publisher's clock falls behind; if the
* publisher's clock is ahead, subsequent transactions will naturally bear
* later commit timestamps, conforming to the design outlined atop of
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 f566e23f785..6f39a713128 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 2acfe736a94..711830b88c8 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -284,10 +289,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
v40-0006-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v40-0006-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 5dabcc3f9e45b0894ec223e1ab9c02d917c7132f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 18:43:49 +0800
Subject: [PATCH v40 6/6] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 63 +-------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 81 +++++++----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 13 +--
8 files changed, 120 insertions(+), 163 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 7b8f92c1a69..4c8b11612b9 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on the
- * publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option when
- * the subscription is enabled to prevent race conditions arising from the
- * new option value being acknowledged asynchronously by the launcher and
- * apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,22 +1353,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the subscription
- * has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker processes
- * have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e4426fa0153..473ac99fb43 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -308,8 +308,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,13 +327,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -457,10 +453,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1181,7 +1175,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1254,8 +1248,7 @@ ApplyLauncherMain(Datum main_arg)
logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo);
+ DSM_HANDLE_INVALID);
}
else
{
@@ -1324,7 +1317,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1333,13 +1326,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1359,10 +1346,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1461,19 +1452,48 @@ CreateConflictDetectionSlot(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index 1591c1c99d4..8e1e8762f62 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -609,8 +609,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
hentry->last_start_time = now;
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6c52280a212..41593ea02e0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -309,8 +309,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -483,8 +483,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4206,6 +4204,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4233,17 +4233,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4251,7 +4254,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4360,7 +4363,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4452,22 +4455,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4480,7 +4477,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4506,7 +4503,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4525,7 +4522,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4560,7 +4557,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4717,6 +4714,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 was changed. 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 ||
@@ -4726,7 +4732,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,
@@ -5406,30 +5413,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 711830b88c8..9156b3ce276 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,19 +228,12 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
+# Test the WARNING and NOTICE messages related to retain_conflict_info during
+# subscription DDLs
##################################################
-# Alter failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
ok( $stderr =~
/WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
--
2.30.0.windows.2
On Mon, Jun 23, 2025 at 4:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V40 patch set
Thanks for the patches. Few comments:
1)
In get_subscription_info(), we are doing COUNT of rci-subscriptions
using below query:
SELECT count(*) AS nsub, COUNT(CASE WHEN subretainconflictinfo THEN 1
END) AS retain_conflict_info FROM pg_catalog.pg_subscription;
And then we are doing:
cluster->sub_retain_conflict_info = (strcmp(PQgetvalue(res, 0,
i_retain_conflict_info), "1") == 0);
i.e. get the value and compare with "1". If the count of such subs is
say 2, won't it fail and will set sub_retain_conflict_info as 0?
2)
create_logical_replication_slots(void)
{
+ if (!count_old_cluster_logical_slots())
+ return;
+
We shall get rid of count_old_cluster_logical_slots() here as the
caller is checking it already.
3)
We can move the 'old_cluster.sub_retain_conflict_info' check from
create_conflict_detection_slot() to its caller. Then it will be more
informative and consistent with how we check migrate_logical_slots
outside of create_conflict_detection_slot()
thanks
Shveta
On Tue, Jun 24, 2025 at 6:22 PM shveta malik wrote:
On Mon, Jun 23, 2025 at 4:20 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V40 patch set
Thanks for the patches. Few comments:
1)
In get_subscription_info(), we are doing COUNT of rci-subscriptions using
below query:
SELECT count(*) AS nsub, COUNT(CASE WHEN subretainconflictinfo THEN 1
END) AS retain_conflict_info FROM pg_catalog.pg_subscription;And then we are doing:
cluster->sub_retain_conflict_info = (strcmp(PQgetvalue(res, 0,
i_retain_conflict_info), "1") == 0);i.e. get the value and compare with "1". If the count of such subs is say 2,
won't it fail and will set sub_retain_conflict_info as 0?
Right, it could return wrong results. I have changed it to count(*) xx > 0
so that it can return directly boolean value.
2) create_logical_replication_slots(void) { + if (!count_old_cluster_logical_slots()) + return; +We shall get rid of count_old_cluster_logical_slots() here as the caller is
checking it already.
Removed.
3)
We can move the 'old_cluster.sub_retain_conflict_info' check from
create_conflict_detection_slot() to its caller. Then it will be more informative
and consistent with how we check migrate_logical_slots outside of
create_conflict_detection_slot()
Moved.
Here is the V41 patch set which includes the following changes:
0001:
* Rebased due to recent commit fd51941.
* Addressed the comments above.
* Improved some documentation stuff.
* Improved the status message when creating
conflict detection slot in pg_upgrade
0002:
No change
0003:
No change
0004:
No change
0005:
No change
0006:
Rebased due to recent commit fd51941.
Best Regards,
Hou zj
Attachments:
v41-0006-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v41-0006-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 6e2dbc28c992cd73286fe5d52b2c3736f83ad0f4 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:29:23 +0800
Subject: [PATCH v41 6/6] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 63 +-------------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 ++++++++++++-------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 81 +++++++----------
src/include/replication/worker_internal.h | 29 ++++++-
src/test/subscription/t/035_conflicts.pl | 13 +--
8 files changed, 120 insertions(+), 163 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 7b8f92c1a69..4c8b11612b9 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on the
- * publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option when
- * the subscription is enabled to prevent race conditions arising from the
- * new option value being acknowledged asynchronously by the launcher and
- * apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,22 +1353,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the subscription
- * has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker processes
- * have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 5261696aaa1..ee8a47fc820 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1194,7 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1267,8 +1261,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1348,7 +1341,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1357,13 +1350,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1383,10 +1370,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1485,19 +1476,48 @@ CreateConflictDetectionSlot(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 14529c65aaf..ba3ede25f7d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -302,8 +302,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -476,8 +476,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4199,6 +4197,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4226,17 +4226,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4244,7 +4247,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4353,7 +4356,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4445,22 +4448,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4473,7 +4470,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4499,7 +4496,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4518,7 +4515,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4553,7 +4550,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4710,6 +4707,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 was changed. 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 ||
@@ -4719,7 +4725,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,
@@ -5399,30 +5406,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 711830b88c8..9156b3ce276 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,19 +228,12 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Test that the retain_conflict_info option cannot be changed for enabled
-# subscriptions.
+# Test the WARNING and NOTICE messages related to retain_conflict_info during
+# subscription DDLs
##################################################
-# Alter failover for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
ok( $stderr =~
/WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
--
2.30.0.windows.2
v41-0001-Retain-the-information-useful-for-detecting-conf.patchapplication/octet-stream; name=v41-0001-Retain-the-information-useful-for-detecting-conf.patchDownload
From ab57be9af473735ee3956c71d4647f10203f19ac Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v41 1/6] Retain the information useful for detecting conflicts
in logical replication
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 and delete_origin_differs
conflicts. These detections require comparing the origin data of local tuples
with remote ones. If the origin data is removed prematurely due to vacuum
freeze, it is impossible to detect these conflicts.
This patch adds a subscription option retain_conflict_info, 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.
Each apply worker for a subscription with retain_conflict_info enabled will
maintain the non-removable transaction ID in the shared memory following the
steps described above. Additionally, the logical replication launcher will
create and maintain a replication slot named pg_conflict_detection if any local
subscription has the retain_conflict_info option enabled. The launcher will
periodically collects the oldest_nonremovable_xid from all apply workers,
computes the minimum transaction ID, advances the xmin value of the replication
slot if it precedes the computed value.
Enabling retain_conflict_info is prohibited if the publisher is currently
in recovery mode (operating as a standby server).
During the upgrade, if any subscription on the old cluster has
retain_conflict_info enabled, create a conflict detection slot to protect
tuples that may be deleted when the new cluster starts.
Bump catalog version
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 220 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 624 +++++++++++++++++-
src/backend/replication/slot.c | 43 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/tools/pgindent/typedefs.list | 2 +
44 files changed, 1819 insertions(+), 184 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 224d4fe5a9f..6fc11f43b7a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29549,7 +29549,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29593,7 +29595,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29623,6 +29627,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 137ffc8d0b7..eb20dbcca4b 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4aec73bcc6b..7b8f92c1a69 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on the
+ * publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the subscription
+ * has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker processes
+ * have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 14d8efbd25b..727508c0894 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after a
+ * restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1184,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1206,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1258,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1279,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1320,155 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ 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();
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on the
+ * publisher that require the data preceding the slot's xmin should have
+ * already been applied and flushed on the subscriber before the xmin is
+ * advanced. So, even if the slot's xmin regresses after a restart, it will
+ * be advanced again in the next cycle. Therefore, no data required for
+ * conflict detection will be prematurely removed.
+ */
+ return;
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..49029638c47 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +391,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +438,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3656,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3735,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3763,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3780,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3817,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3856,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3927,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3964,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;
@@ -3920,6 +4041,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably
+ * even for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5249,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5429,13 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5798,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c11e588d632..f07ef2b87a3 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -564,6 +597,12 @@ ReplicationSlotAcquire(const char *name, bool nowait, bool error_if_invalid)
Assert(name != NULL);
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
retry:
Assert(MyReplicationSlot == NULL);
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index db944ec2230..e5cfa5992f4 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 7417eab6aef..945b3fce670 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 81865cd3e48..de8f800466a 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,48 +1943,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1997,12 +2029,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2158,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..579c9fa717a 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) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..c6dbfc6da50 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to
+ * WALs as it only retains the dead tuples. It is created here for
+ * consistency. Note that the new conflict detection slot uses the latest
+ * transaction ID as xmin, so it cannot protect dead tuples that existed
+ * before the upgrade. Additionally, commit timestamps and origin data
+ * are not preserved during the upgrade. So, even after creating the slot,
+ * the upgraded subscriber may be unable to detect conflicts or log
+ * relevant commit timestamps and origins when applying changes from the
+ * publisher occurred before the upgrade especially if those changes were
+ * not replicated. It can only protect tuples that might be deleted after
+ * the new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 c545abf6581..0d820720650 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -198,8 +246,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -209,7 +258,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -268,7 +317,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -291,7 +341,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -301,15 +352,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -328,6 +380,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..cd7cb2115c2 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,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 908eef97c6e..9b210fb2ec7 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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_proc.dat b/src/include/catalog/pg_proc.dat
index d3d28a263fa..359f03cd331 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11773,6 +11773,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v41-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v41-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 66ef4d2906f1afd1c9b2d8f81b10f3bc67175709 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 17:53:17 +0800
Subject: [PATCH v41 2/6] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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, 332 insertions(+), 28 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 727508c0894..a988969636e 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1186,6 +1193,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1218,7 +1226,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1231,7 +1240,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1280,14 +1289,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements:
+ * - Invalidate the slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it.
+ * - Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information.
+ * - if required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1328,17 +1342,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1349,8 +1367,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1379,6 +1411,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1469,6 +1507,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information useful for
+ * conflict detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Is current process the logical replication launcher?
*/
@@ -1512,7 +1581,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;
@@ -1589,6 +1658,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 49029638c47..b80e028eb9d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -448,6 +448,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3860,7 +3862,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4136,6 +4139,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4276,6 +4283,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4357,6 +4371,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4382,7 +4415,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4402,12 +4438,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4415,9 +4460,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4450,6 +4550,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index f07ef2b87a3..935828b08ac 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"},
};
/*
@@ -1663,6 +1664,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();
}
@@ -1781,6 +1787,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;
}
@@ -2051,6 +2063,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 359f03cd331..f566e23f785 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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
--
2.30.0.windows.2
v41-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v41-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 2b5c82677f8e40b9071180c78b708260fa084678 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v41 3/6] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index a988969636e..5261696aaa1 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1391,6 +1391,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b80e028eb9d..a16c7d3a973 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -476,6 +476,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4139,10 +4141,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4422,6 +4420,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4429,6 +4446,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4470,9 +4488,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4501,19 +4518,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v41-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchapplication/octet-stream; name=v41-0004-Add-a-tap-test-to-verify-the-management-of-the-n.patchDownload
From 3bb440440872e03b104ed55da1c97e514e52a7b2 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@cn.fujitsu.com>
Date: Thu, 24 Apr 2025 11:21:48 +0800
Subject: [PATCH v41 4/6] Add a tap test to verify the management of the new
replication slot
---
src/test/subscription/t/035_conflicts.pl | 181 ++++++++++++++++++++++-
1 file changed, 179 insertions(+), 2 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..2acfe736a94 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 @@ $node_publisher->start;
# 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,181 @@ $node_subscriber->wait_for_log(
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');
+
+##################################################
+# Test that the retain_conflict_info option cannot be changed for enabled
+# subscriptions.
+##################################################
+
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+($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();
--
2.30.0.windows.2
v41-0005-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v41-0005-Support-the-conflict-detection-for-update_delete.patchDownload
From 7f464cd1cb5823595f8c47db19a7574b21e1eb6f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 20 Jun 2025 18:28:35 +0800
Subject: [PATCH v41 5/6] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 58 ++++++---
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 25 +++-
15 files changed, 278 insertions(+), 42 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 a16c7d3a973..14529c65aaf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2809,17 +2809,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. */
@@ -3157,18 +3168,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;
}
@@ -4051,10 +4073,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4093,10 +4115,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4131,7 +4153,7 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably
+ * subscription by the main apply worker to detect update_deleted conflict
* even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
@@ -4337,7 +4359,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily
+ * required to detect update_delete conflict. This check primarily
* addresses scenarios where the publisher's clock falls behind; if the
* publisher's clock is ahead, subsequent transactions will naturally bear
* later commit timestamps, conforming to the design outlined atop of
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 f566e23f785..6f39a713128 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 2acfe736a94..711830b88c8 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -284,10 +289,26 @@ ok( $stderr =~
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;");
--
2.30.0.windows.2
Here is the V41 patch set which includes the following changes:
Thanks for the patches. Few trivial things:
1)
In ReplicationSlotAcquire(), does it make more sense to move the error
after checking the slot's existence first? If a user is trying to use
a slot which does not exist, he should first get that error instead of
'slot is reserved' error.
2)
When max_replication_slots limit is reached and user is trying to
enable rci for the first time, launcher will give error in log file:
ERROR: all replication slots are in use
HINT: Free one or increase "max_replication_slots".
LOG: background worker "logical replication launcher" (PID 13147)
exited with exit code 1
It is not clear from this message as to what launcher was actually
trying to create. A log-msg in CreateConflictDetectionSlot() saying
"Creating conflict-detection slot" may help here.
thanks
Shveta
On Wed, Jun 25, 2025 at 8:38 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V41 patch set which includes the following changes:
Few comments on 0004
===================
1.
+
+# 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");
+
Why use an indirect way to verify that the vacuum can now remove rows?
Even if we want to check that the conflict slot is getting updated
properly, we should verify that the vacuum has removed the deleted
rows. Also, please improve comments for this test, as it is not very
clear why you are expecting the latest xid value of conflict_slot.
2.
+# Alter failover for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled
subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed
until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if
retain_conflict_info is enabled");
+
+# Alter failover for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed
until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
In all places, the comments use failover as an option name, whereas it
is testing retain_conflict_info.
3. It is better to merge the 0004 into 0001 as it tests the core part
of the functionality added by 0001.
--
With Regards,
Amit Kapila.
On Wed, Jun 25, 2025 at 2:57 PM shveta malik wrote:
Here is the V41 patch set which includes the following changes:
Thanks for the patches. Few trivial things:
1)
In ReplicationSlotAcquire(), does it make more sense to move the error after
checking the slot's existence first? If a user is trying to use a slot which does
not exist, he should first get that error instead of 'slot is reserved' error.2)
When max_replication_slots limit is reached and user is trying to enable rci for
the first time, launcher will give error in log file:ERROR: all replication slots are in use
HINT: Free one or increase "max_replication_slots".
LOG: background worker "logical replication launcher" (PID 13147) exited
with exit code 1It is not clear from this message as to what launcher was actually trying to
create. A log-msg in CreateConflictDetectionSlot() saying "Creating
conflict-detection slot" may help here.
Thanks for the comments. All of them look good to me and
have been addressed in V42.
Best Regards,
Hou zj
On Wed, Jun 25, 2025 at 7:27 PM Amit Kapila wrote:
On Wed, Jun 25, 2025 at 8:38 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V41 patch set which includes the following changes:
Few comments on 0004 =================== 1. + +# 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"); +Why use an indirect way to verify that the vacuum can now remove rows?
Even if we want to check that the conflict slot is getting updated properly, we
should verify that the vacuum has removed the deleted rows. Also, please
improve comments for this test, as it is not very clear why you are expecting the
latest xid value of conflict_slot.
I agree that testing VACUUM is straightforward. But I think there is a gap
between applying remote changes and updating slot.xmin in the launcher.
Therefore, it's necessary to wait for the launcher to update the slot before
testing whether VACUUM can remove the dead tuple.
I have improved the comments and added the VACUUM test as
suggested after the slot.xmin test.
2. +# Alter failover for enabled subscription my ($cmdret, $stdout, +$stderr) = $node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB +SET (retain_conflict_info = true)"); ok( $stderr =~ + /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/, + "altering retain_conflict_info is not allowed for enabled + subscription"); + +# Disable the subscription +($cmdret, $stdout, $stderr) = $node_A->psql('postgres', "ALTER +SUBSCRIPTION $subname_AB DISABLE;"); ok( $stderr =~ + /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/, + "A warning is raised on disabling the subscription if retain_conflict_info is enabled"); + +# Alter failover for disabled subscription ($cmdret, $stdout, $stderr) += $node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB SET +(retain_conflict_info = true);"); ok( $stderr =~ + /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/, + "altering retain_conflict_info is allowed for disabled subscription");In all places, the comments use failover as an option name, whereas it is
testing retain_conflict_info.
Changed.
3. It is better to merge the 0004 into 0001 as it tests the core part of the
functionality added by 0001.
Merged.
Here is the V41 patch set which includes the following changes:
0001:
* ran pgindent
* Merge the original v41-0004 tap-test.
* Addressed the comments above.
* Addressed the Shveta's comments[1]/messages/by-id/CAJpy0uAg1mTcy00nR=VAx1nTJYRkQF84YOY4_YKh8L53A1t6sA@mail.gmail.com.
0002:
* ran pgindent
0003:
* ran pgindent
0004:
* ran pgindent
0005:
* ran pgindent
[1]: /messages/by-id/CAJpy0uAg1mTcy00nR=VAx1nTJYRkQF84YOY4_YKh8L53A1t6sA@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v42-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v42-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From 32ef6d914ff5b2eee690c5d1a3e850ed111e20fd Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v42] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 223 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 625 +++++++++++++++++-
src/backend/replication/slot.c | 43 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 185 +++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2006 insertions(+), 186 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 224d4fe5a9f..6fc11f43b7a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29549,7 +29549,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29593,7 +29595,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29623,6 +29627,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4aec73bcc6b..9e47909a30e 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 14d8efbd25b..d24325326f4 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after
+ * a restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1184,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1206,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1258,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1279,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1320,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..ee9a5542cb9 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +391,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +438,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3656,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3735,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3763,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3780,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3817,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3856,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3927,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3964,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;
@@ -3920,6 +4041,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5249,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5429,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5799,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c11e588d632..7a58ce9485b 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,12 @@ retry:
name)));
}
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index db944ec2230..e5cfa5992f4 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 7417eab6aef..945b3fce670 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 81865cd3e48..de8f800466a 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,48 +1943,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1997,12 +2029,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2158,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..579c9fa717a 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) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 c545abf6581..0d820720650 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -198,8 +246,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -209,7 +258,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -268,7 +317,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -291,7 +341,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -301,15 +352,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -328,6 +380,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..cd7cb2115c2 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,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 908eef97c6e..9b210fb2ec7 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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_proc.dat b/src/include/catalog/pg_proc.dat
index d3d28a263fa..359f03cd331 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11773,6 +11773,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..65a3f1d2cd5 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 @@ $node_publisher->start;
# 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,185 @@ $node_subscriber->wait_for_log(
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");
+
+##################################################
+# Check that the retain_conflict_info option cannot be changed for enabled
+# subscriptions, and validate the WARNING and NOTICE messages during
+# subscription DDLs.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+ok( $stderr =~
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on disabling the subscription if retain_conflict_info is enabled");
+
+# Alter retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+###############################################################################
+# 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);
+
+my $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);
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v42-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v42-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 22383a202cac87bd5e1a85d0f4b94d5dfd8efa37 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 09:43:51 +0800
Subject: [PATCH v42 2/5] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 9 +-
15 files changed, 340 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 d24325326f4..403137282fd 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1186,6 +1193,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1218,7 +1226,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1231,7 +1240,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1280,14 +1289,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements: - Invalidate the
+ * slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it. - Drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain the conflict information. - if required,
+ * advance the slot's xmin to protect deleted tuples required for the
+ * conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1328,17 +1342,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1349,8 +1367,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1379,6 +1411,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1435,6 +1473,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1515,7 +1584,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;
@@ -1592,6 +1661,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 ee9a5542cb9..f833d995c50 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -448,6 +448,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3860,7 +3862,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4136,6 +4139,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4276,6 +4283,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4357,6 +4371,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4382,7 +4415,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4402,12 +4438,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4415,9 +4460,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4450,6 +4550,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 7a58ce9485b..7563f31604b 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"},
};
/*
@@ -1663,6 +1664,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();
}
@@ -1781,6 +1787,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;
}
@@ -2051,6 +2063,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 359f03cd331..f566e23f785 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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 920d8014f0b..760f664bb47 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -217,6 +217,13 @@ ok( $node_B->poll_query_until(
),
"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 the retain_conflict_info option cannot be changed for enabled
# subscriptions, and validate the WARNING and NOTICE messages during
@@ -256,7 +263,7 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.31.1
v42-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v42-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From ab9faa437544d037a46990269cf56d420bbce4a4 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v42 3/5] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 403137282fd..5ed280d57f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1391,6 +1391,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f833d995c50..0528e82f6da 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -476,6 +476,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4139,10 +4141,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4422,6 +4420,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4429,6 +4446,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4470,9 +4488,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4501,19 +4518,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.31.1
v42-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v42-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 70de1296b509ef5ed70c55dd9fed814ba49ee559 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:45:13 +0800
Subject: [PATCH v42 4/5] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 66 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 25 +++-
15 files changed, 282 insertions(+), 46 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 0528e82f6da..b5eb608d21e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2809,17 +2809,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. */
@@ -3157,18 +3168,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;
}
@@ -4051,10 +4073,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4093,10 +4115,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4131,8 +4153,8 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4337,10 +4359,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop of
+ * required to detect update_delete reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
* maybe_advance_nonremovable_xid().
*
* XXX Consider waiting for the publisher's clock to catch up with the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 f566e23f785..6f39a713128 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 760f664bb47..51ad9b40258 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -257,6 +260,8 @@ $node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
###############################################################################
# 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
@@ -285,10 +290,26 @@ ok( $stderr =~
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;");
--
2.31.1
v42-0005-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v42-0005-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 197f319cf26f0f2265bb90a42df8ecfa91eb3927 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:47:48 +0800
Subject: [PATCH v42 5/5] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 70 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 +++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 101 +++++++-----------
src/include/replication/logicalworker.h | 3 +-
src/include/replication/worker_internal.h | 29 ++++-
src/test/subscription/t/035_conflicts.pl | 17 +--
9 files changed, 132 insertions(+), 185 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 9e47909a30e..ca2fe71e781 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,7 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
- !opts.enabled, WARNING);
+ !opts.enabled);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on
- * the publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option
- * when the subscription is enabled to prevent race conditions arising
- * from the new option value being acknowledged asynchronously by the
- * launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,28 +1353,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the
- * subscription has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker
- * processes have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
CheckSubConflictInfoRetention(opts.retainconflictinfo,
- true, !sub->enabled, NOTICE);
+ true, !sub->enabled);
/*
* Notify the launcher to manage the replication slot for
@@ -1458,8 +1399,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* comments atop CheckSubConflictInfoRetention() for details.
*/
CheckSubConflictInfoRetention(sub->retainconflictinfo,
- opts.enabled, !opts.enabled,
- WARNING);
+ opts.enabled, !opts.enabled);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 5ed280d57f2..414c12797cd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1194,7 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1267,8 +1261,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1348,7 +1341,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1357,13 +1350,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1383,10 +1370,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1451,19 +1442,48 @@ acquire_conflict_slot_if_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b5eb608d21e..960192cc660 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -302,8 +302,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -476,8 +476,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4199,6 +4197,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4226,17 +4226,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4244,7 +4247,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4353,7 +4356,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4445,22 +4448,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4473,7 +4470,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4499,7 +4496,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4518,7 +4515,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4553,7 +4550,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4710,6 +4707,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 was changed. 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 ||
@@ -4719,7 +4725,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,
@@ -5399,30 +5406,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5585,7 +5568,7 @@ DisableSubscriptionAndExit(void)
* context.
*/
CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
- true, WARNING);
+ true);
proc_exit(0);
}
@@ -5963,18 +5946,13 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_conflict_info for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING if the subscription is disabled. Do not raise an ERROR since
+ * as long as the subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
- bool sub_disabled, int elevel_for_sub_disabled)
+ bool sub_disabled)
{
- Assert(elevel_for_sub_disabled == NOTICE ||
- elevel_for_sub_disabled == WARNING);
-
if (!retain_conflict_info)
return;
@@ -5992,10 +5970,9 @@ CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
"track_commit_timestamp"));
if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_conflict_info") : 0);
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
}
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index ebbb3ef09c7..2d006e7888e 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -32,7 +32,6 @@ extern void AtEOXact_LogicalRepWorkers(bool isCommit);
extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
bool check_guc,
- bool sub_disabled,
- int elevel_for_sub_disabled);
+ bool sub_disabled);
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 51ad9b40258..d2284b857be 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -228,20 +228,11 @@ $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Check that the retain_conflict_info option cannot be changed for enabled
-# subscriptions, and validate the WARNING and NOTICE messages during
-# subscription DDLs.
+# Validate the WARNING and NOTICE messages during subscription DDLs.
##################################################
-# Alter retain_conflict_info for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
# Disable the subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB DISABLE;");
ok( $stderr =~
/WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
@@ -251,8 +242,8 @@ ok( $stderr =~
($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
ok( $stderr =~
- /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
- "altering retain_conflict_info is allowed for disabled subscription");
+ /WARNING: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "A warning is raised on enabling retain_conflict_info for disabled subscription");
# Re-enable the subscription
$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
--
2.31.1
On Thu, Jun 26, 2025 at 8:31 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Thanks for the comments. All of them look good to me and
have been addressed in V42.
Thank You for the patches. Few comments.
t/035_conflicts.pl:
1)
Both the subscriptions subname_BA and subname_AB have rci enabled
during CREATE sub itself. And later in the second test, we are trying
to enable rci of subname_AB to test WARNING and NOTICE, but rci is
already enabled. Shall we have one CREATE sub with rci enabled while
another CREATE sub with default rci. And then we try to enable rci of
the second sub later and check pg_conflict_detection slot has been
created once we enabled rci. This way, it will cover more scenarios.
2)
+$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);
Can you please help me understand why we are doing wait_for_catchup
here? Do we want DELETE to be replicated from A to B? IMO, this step
is not essential for our test as we have node_A->poll_query until
xmin = $next_xid in pg_conflict_detection and that should suffice to
ensure both DELETE and UPDATE are replicated from one to other.
thanks
Shveta
On Thu, Jun 26, 2025 at 4:28 PM shveta malik wrote:
On Thu, Jun 26, 2025 at 8:31 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Thanks for the comments. All of them look good to me and
have been addressed in V42.Thank You for the patches. Few comments.
t/035_conflicts.pl:
1)
Both the subscriptions subname_BA and subname_AB have rci enabled
during CREATE sub itself. And later in the second test, we are trying
to enable rci of subname_AB to test WARNING and NOTICE, but rci is
already enabled. Shall we have one CREATE sub with rci enabled while
another CREATE sub with default rci. And then we try to enable rci of
the second sub later and check pg_conflict_detection slot has been
created once we enabled rci. This way, it will cover more scenarios.
Agreed and changed as suggested. I removed the test for WARNING since the
message is the same as the NOITCE and it seems not worthwhile to disable
the subscription again to verify one message.
2) +$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);Can you please help me understand why we are doing wait_for_catchup
here? Do we want DELETE to be replicated from A to B? IMO, this step
is not essential for our test as we have node_A->poll_query until
xmin = $next_xid in pg_conflict_detection and that should suffice to
ensure both DELETE and UPDATE are replicated from one to other.
I think this step belongs to a later patch to ensure the DELETE operation is
replicated to Node B, allowing us to verify the `delete_origin_differ`
conflicts detected there. So, I moved it to the later patches.
Here is the V43 patch set which includes the following changes:
0001:
* Addressed the comments above.
0002:
No change.
0003:
No change.
0004:
* Moved some tests from 0001 to here.
0005:
No change.
Best Regards,
Hou zj
Attachments:
v43-0005-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v43-0005-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 2ba5d20700b121060cc72799da9a60797a26d565 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Jun 2025 10:18:22 +0800
Subject: [PATCH v43 5/5] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 70 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 +++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 101 +++++++-----------
src/include/replication/logicalworker.h | 3 +-
src/include/replication/worker_internal.h | 29 ++++-
src/test/subscription/t/035_conflicts.pl | 26 +----
9 files changed, 132 insertions(+), 194 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 9e47909a30e..ca2fe71e781 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,7 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
- !opts.enabled, WARNING);
+ !opts.enabled);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on
- * the publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option
- * when the subscription is enabled to prevent race conditions arising
- * from the new option value being acknowledged asynchronously by the
- * launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,28 +1353,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the
- * subscription has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker
- * processes have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
CheckSubConflictInfoRetention(opts.retainconflictinfo,
- true, !sub->enabled, NOTICE);
+ true, !sub->enabled);
/*
* Notify the launcher to manage the replication slot for
@@ -1458,8 +1399,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* comments atop CheckSubConflictInfoRetention() for details.
*/
CheckSubConflictInfoRetention(sub->retainconflictinfo,
- opts.enabled, !opts.enabled,
- WARNING);
+ opts.enabled, !opts.enabled);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 5ed280d57f2..414c12797cd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1194,7 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1267,8 +1261,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1348,7 +1341,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1357,13 +1350,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1383,10 +1370,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1451,19 +1442,48 @@ acquire_conflict_slot_if_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b5eb608d21e..960192cc660 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -302,8 +302,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -476,8 +476,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4199,6 +4197,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4226,17 +4226,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4244,7 +4247,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4353,7 +4356,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4445,22 +4448,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4473,7 +4470,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4499,7 +4496,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4518,7 +4515,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4553,7 +4550,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4710,6 +4707,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 was changed. 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 ||
@@ -4719,7 +4725,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,
@@ -5399,30 +5406,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5585,7 +5568,7 @@ DisableSubscriptionAndExit(void)
* context.
*/
CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
- true, WARNING);
+ true);
proc_exit(0);
}
@@ -5963,18 +5946,13 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_conflict_info for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING if the subscription is disabled. Do not raise an ERROR since
+ * as long as the subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
- bool sub_disabled, int elevel_for_sub_disabled)
+ bool sub_disabled)
{
- Assert(elevel_for_sub_disabled == NOTICE ||
- elevel_for_sub_disabled == WARNING);
-
if (!retain_conflict_info)
return;
@@ -5992,10 +5970,9 @@ CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
"track_commit_timestamp"));
if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_conflict_info") : 0);
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
}
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index ebbb3ef09c7..2d006e7888e 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -32,7 +32,6 @@ extern void AtEOXact_LogicalRepWorkers(bool isCommit);
extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
bool check_guc,
- bool sub_disabled,
- int elevel_for_sub_disabled);
+ bool sub_disabled);
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 58d510b12d7..cf68397785f 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -218,31 +218,13 @@ my $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Check that the retain_conflict_info option can be enabled only for disabled
-# subscriptions. Validate the NOTICE message during the subscription DDL, and
-# ensure the conflict detection slot is created upon enabling the
+# Check that the conflict detection slot is created after enabling
# retain_conflict_info option.
##################################################
-# Alter retain_conflict_info for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
-# Disable the subscription
-$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
-
-# Enable retain_conflict_info for disabled subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+# Enable retain_conflict_info for the subscription
+$node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
-ok( $stderr =~
- /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
- "altering retain_conflict_info is allowed for disabled subscription");
-
-# Re-enable the subscription
-$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
# Confirm that the conflict detection slot is created on Node A and the xmin
# value is valid.
@@ -279,7 +261,7 @@ 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;");
-($cmdret, $stdout, $stderr) = $node_A->psql(
+my ($cmdret, $stdout, $stderr) = $node_A->psql(
'postgres', qq(VACUUM (verbose) public.tab;)
);
--
2.30.0.windows.2
v43-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v43-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From e7f0d0cddbe137947023e52abf27f975faafa32b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v43 1/5] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 223 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 625 +++++++++++++++++-
src/backend/replication/slot.c | 43 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 179 ++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2000 insertions(+), 186 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 224d4fe5a9f..6fc11f43b7a 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29549,7 +29549,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29593,7 +29595,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29623,6 +29627,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4aec73bcc6b..9e47909a30e 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 14d8efbd25b..d24325326f4 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,14 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Attempts to acquire the slot designated for conflict detection at the
+ * start. This ensures that if the launcher loses track of the slot after
+ * a restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1184,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1206,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1258,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1279,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1320,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..ee9a5542cb9 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +391,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +438,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3656,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3735,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3763,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3780,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3817,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3856,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3927,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3964,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;
@@ -3920,6 +4041,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5249,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5429,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5799,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index c11e588d632..7a58ce9485b 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,12 @@ retry:
name)));
}
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1937997ea67..02d3746399e 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 81865cd3e48..de8f800466a 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,48 +1943,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1997,12 +2029,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2158,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..579c9fa717a 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) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 c545abf6581..0d820720650 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -87,6 +87,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -198,8 +246,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -209,7 +258,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -268,7 +317,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -291,7 +341,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -301,15 +352,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -328,6 +380,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..cd7cb2115c2 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,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 908eef97c6e..9b210fb2ec7 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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_proc.dat b/src/include/catalog/pg_proc.dat
index d3d28a263fa..359f03cd331 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11773,6 +11773,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 2a7a8239a29..8921cb5ef1e 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 @@ $node_publisher->start;
# 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,179 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v43-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v43-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 85e93afd38039f7ba2efdf77ff7f9208540f371c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 15:08:55 +0800
Subject: [PATCH v43 2/5] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 341 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 d24325326f4..403137282fd 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1186,6 +1193,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1218,7 +1226,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1231,7 +1240,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1280,14 +1289,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements: - Invalidate the
+ * slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it. - Drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain the conflict information. - if required,
+ * advance the slot's xmin to protect deleted tuples required for the
+ * conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1328,17 +1342,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1349,8 +1367,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1379,6 +1411,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1435,6 +1473,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1515,7 +1584,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;
@@ -1592,6 +1661,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 ee9a5542cb9..f833d995c50 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -448,6 +448,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3860,7 +3862,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4136,6 +4139,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4276,6 +4283,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4357,6 +4371,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4382,7 +4415,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4402,12 +4438,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4415,9 +4460,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4450,6 +4550,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 7a58ce9485b..7563f31604b 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"},
};
/*
@@ -1663,6 +1664,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();
}
@@ -1781,6 +1787,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;
}
@@ -2051,6 +2063,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 f04bfedb2fd..bd4e73099af 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 359f03cd331..f566e23f785 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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 8921cb5ef1e..6399477ae53 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -210,6 +210,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $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 the retain_conflict_info option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -245,6 +249,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$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');
+
###############################################################################
# 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.
@@ -254,7 +262,7 @@ ok( $node_A->poll_query_until(
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.30.0.windows.2
v43-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v43-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 922b7d4e54806208be1bdf0503a2d2f944ff26aa Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v43 3/5] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 403137282fd..5ed280d57f2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1391,6 +1391,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f833d995c50..0528e82f6da 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -476,6 +476,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4139,10 +4141,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4422,6 +4420,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4429,6 +4446,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4470,9 +4488,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4501,19 +4518,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v43-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v43-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From ccca7fa77375b099385f9806d456ec7e501045d1 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:45:13 +0800
Subject: [PATCH v43 4/5] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 66 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 31 ++++-
15 files changed, 288 insertions(+), 46 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 0528e82f6da..b5eb608d21e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2809,17 +2809,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. */
@@ -3157,18 +3168,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;
}
@@ -4051,10 +4073,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4093,10 +4115,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4131,8 +4153,8 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4337,10 +4359,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop of
+ * required to detect update_delete reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
* maybe_advance_nonremovable_xid().
*
* XXX Consider waiting for the publisher's clock to catch up with the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 f566e23f785..6f39a713128 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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 6399477ae53..58d510b12d7 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -149,7 +149,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -256,6 +259,8 @@ is($result, qq(t), 'worker on node A 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.
+# And check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -269,6 +274,8 @@ is($result, qq(1|1
# 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;");
@@ -280,10 +287,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
--
2.30.0.windows.2
On Fri, Jun 27, 2025 at 7:58 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thu, Jun 26, 2025 at 4:28 PM shveta malik wrote:
On Thu, Jun 26, 2025 at 8:31 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Thanks for the comments. All of them look good to me and
have been addressed in V42.Thank You for the patches. Few comments.
t/035_conflicts.pl:
1)
Both the subscriptions subname_BA and subname_AB have rci enabled
during CREATE sub itself. And later in the second test, we are trying
to enable rci of subname_AB to test WARNING and NOTICE, but rci is
already enabled. Shall we have one CREATE sub with rci enabled while
another CREATE sub with default rci. And then we try to enable rci of
the second sub later and check pg_conflict_detection slot has been
created once we enabled rci. This way, it will cover more scenarios.Agreed and changed as suggested. I removed the test for WARNING since the
message is the same as the NOITCE and it seems not worthwhile to disable
the subscription again to verify one message.
Okay. Agreed.
patch001 looks good to me now.
thanks
Shveta
On Fri, Jun 27, 2025 at 7:58 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V43 patch set which includes the following changes:
Few minor comments:
1.
@@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn +
pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
/The name/The new slot name
2. The server version checks can be modified to 19000 as a new branch
is created now.
3.
This ensures that if the launcher loses track of the slot after
+ * a restart, it will remember to drop the slot when it is no longer
+ * requested by any subscription.
The link between the first part of the sentence before the comma is
not clear with the remaining part of the sentence. How about writing
it as: "Acquire the conflict detection slot at startup to ensure it
can be dropped if no longer needed after a restart."?
--
With Regards,
Amit Kapila.
On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
On Fri, Jun 27, 2025 at 7:58 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is the V43 patch set which includes the following changes:
Few minor comments: 1. @@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset Copies an existing logical replication slot named <parameter>src_slot_name</parameter> to a logical replication slot named <parameter>dst_slot_name</parameter>, optionally changing - the output plugin and persistence. The copied logical slot starts - from the same <acronym>LSN</acronym> as the source logical slot. Both + the output plugin and persistence. The name cannot be + <literal>pg_conflict_detection</literal> as it is reserved for + the conflict detection. The copied logical slot starts from the same + <acronym>LSN</acronym> as the source logical slot. Both/The name/The new slot name
2. The server version checks can be modified to 19000 as a new branch is
created now.3. This ensures that if the launcher loses track of the slot after + * a restart, it will remember to drop the slot when it is no longer + * requested by any subscription.The link between the first part of the sentence before the comma is not clear
with the remaining part of the sentence. How about writing it as: "Acquire the
conflict detection slot at startup to ensure it can be dropped if no longer needed
after a restart."?
Thanks for the comments. I have addressed above in V44 patch set.
Only 0001 is modified in this version.
Best Regards,
Hou zj
Attachments:
v44-0005-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v44-0005-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 49a605e2cc5dbdc0db72791fe942c67fe7a82e6f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Jun 2025 10:18:22 +0800
Subject: [PATCH v44 5/5] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 70 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 +++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 101 +++++++-----------
src/include/replication/logicalworker.h | 3 +-
src/include/replication/worker_internal.h | 29 ++++-
src/test/subscription/t/035_conflicts.pl | 26 +----
9 files changed, 132 insertions(+), 194 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 0cba169e0d1..b3f284b4266 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,7 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
- !opts.enabled, WARNING);
+ !opts.enabled);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on
- * the publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option
- * when the subscription is enabled to prevent race conditions arising
- * from the new option value being acknowledged asynchronously by the
- * launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,28 +1353,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the
- * subscription has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker
- * processes have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
CheckSubConflictInfoRetention(opts.retainconflictinfo,
- true, !sub->enabled, NOTICE);
+ true, !sub->enabled);
/*
* Notify the launcher to manage the replication slot for
@@ -1458,8 +1399,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* comments atop CheckSubConflictInfoRetention() for details.
*/
CheckSubConflictInfoRetention(sub->retainconflictinfo,
- opts.enabled, !opts.enabled,
- WARNING);
+ opts.enabled, !opts.enabled);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4c3ed2c9292..a8413d4abef 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1194,7 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1267,8 +1261,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1348,7 +1341,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1357,13 +1350,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1383,10 +1370,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1451,19 +1442,48 @@ acquire_conflict_slot_if_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b5eb608d21e..960192cc660 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -302,8 +302,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -476,8 +476,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4199,6 +4197,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4226,17 +4226,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4244,7 +4247,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4353,7 +4356,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4445,22 +4448,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4473,7 +4470,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4499,7 +4496,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4518,7 +4515,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4553,7 +4550,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4710,6 +4707,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 was changed. 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 ||
@@ -4719,7 +4725,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,
@@ -5399,30 +5406,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5585,7 +5568,7 @@ DisableSubscriptionAndExit(void)
* context.
*/
CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
- true, WARNING);
+ true);
proc_exit(0);
}
@@ -5963,18 +5946,13 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_conflict_info for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING if the subscription is disabled. Do not raise an ERROR since
+ * as long as the subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
- bool sub_disabled, int elevel_for_sub_disabled)
+ bool sub_disabled)
{
- Assert(elevel_for_sub_disabled == NOTICE ||
- elevel_for_sub_disabled == WARNING);
-
if (!retain_conflict_info)
return;
@@ -5992,10 +5970,9 @@ CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
"track_commit_timestamp"));
if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_conflict_info") : 0);
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
}
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index ebbb3ef09c7..2d006e7888e 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -32,7 +32,6 @@ extern void AtEOXact_LogicalRepWorkers(bool isCommit);
extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
bool check_guc,
- bool sub_disabled,
- int elevel_for_sub_disabled);
+ bool sub_disabled);
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 21407fb325b..bca7ec90300 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -219,31 +219,13 @@ my $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Check that the retain_conflict_info option can be enabled only for disabled
-# subscriptions. Validate the NOTICE message during the subscription DDL, and
-# ensure the conflict detection slot is created upon enabling the
+# Check that the conflict detection slot is created after enabling
# retain_conflict_info option.
##################################################
-# Alter retain_conflict_info for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
-# Disable the subscription
-$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
-
-# Enable retain_conflict_info for disabled subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+# Enable retain_conflict_info for the subscription
+$node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
-ok( $stderr =~
- /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
- "altering retain_conflict_info is allowed for disabled subscription");
-
-# Re-enable the subscription
-$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
# Confirm that the conflict detection slot is created on Node A and the xmin
# value is valid.
@@ -280,7 +262,7 @@ 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;");
-($cmdret, $stdout, $stderr) = $node_A->psql(
+my ($cmdret, $stdout, $stderr) = $node_A->psql(
'postgres', qq(VACUUM (verbose) public.tab;)
);
--
2.30.0.windows.2
v44-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v44-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From 7ae9ca15c45c1e01175af8abf4156e27bf7ca494 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v44] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 26 +-
src/backend/access/transam/xact.c | 12 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 221 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 625 +++++++++++++++++-
src/backend/replication/slot.c | 43 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 179 ++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 1998 insertions(+), 186 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 224d4fe5a9f..c0fbba28798 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29549,7 +29549,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29593,7 +29595,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29623,6 +29627,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29645,8 +29651,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..ca9e349e099 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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 transactions that have acquired commit timestamp
+ * 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 b885513f765..5226fa561b1 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,16 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
/*
* Insert the commit XLOG record.
@@ -1537,7 +1543,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 08f780a2e63..ec4aa9ea7b4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4ff246cd943..0cba169e0d1 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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) < 19000)
+ 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 19"));
+
+ 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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..b69b47fd72c 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1182,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1204,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1256,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1277,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1318,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..ee9a5542cb9 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,62 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH) */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +391,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +438,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3656,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3735,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3763,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3780,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3817,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3856,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3927,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3964,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;
@@ -3920,6 +4041,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5249,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5429,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5799,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index f9fec50ae88..8ead54afa9d 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,12 @@ retry:
name)));
}
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1937997ea67..19517c51a9b 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 81865cd3e48..de8f800466a 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -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();
}
@@ -754,7 +754,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1943,48 +1943,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1997,12 +2029,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2114,6 +2158,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..9436fb7c310 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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 e46f02c6cc6..11e7e21478a 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..dbcf88be945 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ 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 908eef97c6e..9b210fb2ec7 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 <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3729,8 +3730,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_proc.dat b/src/include/catalog/pg_proc.dat
index fb4f7f50350..8d233c9c632 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11776,6 +11776,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..58001aff466 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,179 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v44-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v44-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From fd70ead9ebd28bcec32801bc9899a86d03a061ac Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 15:08:55 +0800
Subject: [PATCH v44 2/5] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 341 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index ec4aa9ea7b4..4ef1ec6e668 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 6f47d811081..ab1bdc4ae2a 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1186,6 +1193,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1218,7 +1226,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1231,7 +1240,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1280,14 +1289,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements: - Invalidate the
+ * slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it. - Drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain the conflict information. - if required,
+ * advance the slot's xmin to protect deleted tuples required for the
+ * conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1328,17 +1342,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1349,8 +1367,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1379,6 +1411,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1435,6 +1473,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1515,7 +1584,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;
@@ -1592,6 +1661,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 ee9a5542cb9..f833d995c50 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -448,6 +448,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3860,7 +3862,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4136,6 +4139,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4276,6 +4283,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4357,6 +4371,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4382,7 +4415,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4402,12 +4438,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4415,9 +4460,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4450,6 +4550,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8ead54afa9d..89bb60e5d55 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"},
};
/*
@@ -1663,6 +1664,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();
}
@@ -1781,6 +1787,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;
}
@@ -2051,6 +2063,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 511dc32d519..3a8ce6b74a8 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 8d233c9c632..815c6582a35 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 58001aff466..890bcdaac57 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -211,6 +211,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $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 the retain_conflict_info option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -246,6 +250,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$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');
+
###############################################################################
# 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.
@@ -255,7 +263,7 @@ ok( $node_A->poll_query_until(
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.30.0.windows.2
v44-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v44-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From c74a7b67a467d1b23140a2c1e31b035cbaa97577 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v44 3/5] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ab1bdc4ae2a..4c3ed2c9292 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1391,6 +1391,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f833d995c50..0528e82f6da 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -476,6 +476,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4139,10 +4141,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4422,6 +4420,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4429,6 +4446,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4470,9 +4488,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4501,19 +4518,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v44-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v44-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 2b99ab8546e223d8c93f7815c89079f9a548d5a9 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:45:13 +0800
Subject: [PATCH v44 4/5] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 66 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 31 ++++-
15 files changed, 288 insertions(+), 46 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4ef1ec6e668..20fcd1a0598 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 0528e82f6da..b5eb608d21e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2809,17 +2809,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. */
@@ -3157,18 +3168,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;
}
@@ -4051,10 +4073,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4093,10 +4115,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4131,8 +4153,8 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4337,10 +4359,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop of
+ * required to detect update_delete reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
* maybe_advance_nonremovable_xid().
*
* XXX Consider waiting for the publisher's clock to catch up with the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 815c6582a35..25c0570de7e 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 890bcdaac57..21407fb325b 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -257,6 +260,8 @@ is($result, qq(t), 'worker on node A 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.
+# And check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -270,6 +275,8 @@ is($result, qq(1|1
# 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;");
@@ -281,10 +288,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
--
2.30.0.windows.2
On Mon, Jun 30, 2025 at 6:59 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
I was looking at 0001, it mostly looks fine to me except this one
case. So here we need to ensure that commits must be acquired after
marking the flag, don't you think we need to ensure strict statement
ordering using memory barrier, or we think it's not required and if so
why?
RecordTransactionCommitPrepared()
{
..
+ 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 transactions that have acquired commit timestamp
+ * 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();
}
--
Regards,
Dilip Kumar
Google
On Tue, Jul 1, 2025 at 10:31 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Jun 30, 2025 at 6:59 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
I was looking at 0001, it mostly looks fine to me except this one
case. So here we need to ensure that commits must be acquired after
marking the flag, don't you think we need to ensure strict statement
ordering using memory barrier, or we think it's not required and if so
why?RecordTransactionCommitPrepared() { .. + 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 transactions that have acquired commit timestamp + * 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(); }
I'm unsure whether the function call inherently acts as a memory
barrier, preventing the compiler from reordering these operations.
This needs to be confirmed.
--
Regards,
Dilip Kumar
Google
On Tue, Jul 1, 2025 at 10:53 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:31 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Jun 30, 2025 at 6:59 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
I was looking at 0001, it mostly looks fine to me except this one
case. So here we need to ensure that commits must be acquired after
marking the flag, don't you think we need to ensure strict statement
ordering using memory barrier, or we think it's not required and if so
why?
Good point. I also think we need a barrier here, but a write barrier
should be sufficient as we want ordering of two store operations.
RecordTransactionCommitPrepared() { .. + 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 transactions that have acquired commit timestamp + * 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(); }I'm unsure whether the function call inherently acts as a memory
barrier, preventing the compiler from reordering these operations.
This needs to be confirmed.
As per my understanding, function calls won't be a memory barrier. In
this regard, we need a similar change in RecordTransactionCommit as
well.
--
With Regards,
Amit Kapila.
On Tue, Jul 1, 2025 at 2:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:53 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:31 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Jun 30, 2025 at 6:59 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
I was looking at 0001, it mostly looks fine to me except this one
case. So here we need to ensure that commits must be acquired after
marking the flag, don't you think we need to ensure strict statement
ordering using memory barrier, or we think it's not required and if so
why?Good point. I also think we need a barrier here, but a write barrier
should be sufficient as we want ordering of two store operations.
+1
RecordTransactionCommitPrepared() { .. + 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 transactions that have acquired commit timestamp + * 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(); }I'm unsure whether the function call inherently acts as a memory
barrier, preventing the compiler from reordering these operations.
This needs to be confirmed.As per my understanding, function calls won't be a memory barrier. In
this regard, we need a similar change in RecordTransactionCommit as
well.
Right, we need this in RecordTransactionCommit() as well.
--
Regards,
Dilip Kumar
Google
On Tue, Jul 1, 2025 at 5:07 PM Dilip Kumar wrote:
On Tue, Jul 1, 2025 at 2:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:53 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:31 AM Dilip Kumar <dilipbalaut@gmail.com>
wrote:
On Mon, Jun 30, 2025 at 6:59 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
I was looking at 0001, it mostly looks fine to me except this one
case. So here we need to ensure that commits must be acquired
after marking the flag, don't you think we need to ensure strict
statement ordering using memory barrier, or we think it's not
required and if so why?Good point. I also think we need a barrier here, but a write barrier
should be sufficient as we want ordering of two store operations.+1
RecordTransactionCommitPrepared() { .. + 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 transactions that have acquired commit + timestamp + * 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(); }I'm unsure whether the function call inherently acts as a memory
barrier, preventing the compiler from reordering these operations.
This needs to be confirmed.As per my understanding, function calls won't be a memory barrier. In
this regard, we need a similar change in RecordTransactionCommit as
well.Right, we need this in RecordTransactionCommit() as well.
Thanks for the comments! I also agree that the barrier is needed.
Here is V45 patch set.
I modified 0001, added write barriers, and improved some comments.
Best Regards,
Hou zj
Attachments:
v45-0005-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v45-0005-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From be2781124ee9d1e60853de20ca88652904be1799 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 27 Jun 2025 10:18:22 +0800
Subject: [PATCH v45 5/5] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 70 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 +++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 101 +++++++-----------
src/include/replication/logicalworker.h | 3 +-
src/include/replication/worker_internal.h | 29 ++++-
src/test/subscription/t/035_conflicts.pl | 26 +----
9 files changed, 132 insertions(+), 194 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index cebfb8b85c9..ff226063458 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,7 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
- !opts.enabled, WARNING);
+ !opts.enabled);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on
- * the publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option
- * when the subscription is enabled to prevent race conditions arising
- * from the new option value being acknowledged asynchronously by the
- * launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,28 +1353,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the
- * subscription has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker
- * processes have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
CheckSubConflictInfoRetention(opts.retainconflictinfo,
- true, !sub->enabled, NOTICE);
+ true, !sub->enabled);
/*
* Notify the launcher to manage the replication slot for
@@ -1458,8 +1399,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* comments atop CheckSubConflictInfoRetention() for details.
*/
CheckSubConflictInfoRetention(sub->retainconflictinfo,
- opts.enabled, !opts.enabled,
- WARNING);
+ opts.enabled, !opts.enabled);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 0967fbd1fe5..5133854c9bc 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1192,7 +1186,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1265,8 +1259,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1346,7 +1339,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1355,13 +1348,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1381,10 +1368,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1449,19 +1440,48 @@ acquire_conflict_slot_if_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ca60a668b1d..79a4bea070c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -311,8 +311,8 @@ typedef struct RetainConflictInfoData
* be awaited to complete before
* entering the final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -485,8 +485,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4208,6 +4206,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4235,17 +4235,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4253,7 +4256,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4362,7 +4365,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4454,22 +4457,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4482,7 +4479,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4508,7 +4505,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4527,7 +4524,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4562,7 +4559,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4719,6 +4716,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 was changed. 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 ||
@@ -4728,7 +4734,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,
@@ -5408,30 +5415,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5594,7 +5577,7 @@ DisableSubscriptionAndExit(void)
* context.
*/
CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
- true, WARNING);
+ true);
proc_exit(0);
}
@@ -5972,18 +5955,13 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_conflict_info for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING if the subscription is disabled. Do not raise an ERROR since
+ * as long as the subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
- bool sub_disabled, int elevel_for_sub_disabled)
+ bool sub_disabled)
{
- Assert(elevel_for_sub_disabled == NOTICE ||
- elevel_for_sub_disabled == WARNING);
-
if (!retain_conflict_info)
return;
@@ -6001,10 +5979,9 @@ CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
"track_commit_timestamp"));
if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_conflict_info") : 0);
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
}
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index ebbb3ef09c7..2d006e7888e 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -32,7 +32,6 @@ extern void AtEOXact_LogicalRepWorkers(bool isCommit);
extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
bool check_guc,
- bool sub_disabled,
- int elevel_for_sub_disabled);
+ bool sub_disabled);
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 21407fb325b..bca7ec90300 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -219,31 +219,13 @@ my $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Check that the retain_conflict_info option can be enabled only for disabled
-# subscriptions. Validate the NOTICE message during the subscription DDL, and
-# ensure the conflict detection slot is created upon enabling the
+# Check that the conflict detection slot is created after enabling
# retain_conflict_info option.
##################################################
-# Alter retain_conflict_info for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
-# Disable the subscription
-$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
-
-# Enable retain_conflict_info for disabled subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+# Enable retain_conflict_info for the subscription
+$node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
-ok( $stderr =~
- /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
- "altering retain_conflict_info is allowed for disabled subscription");
-
-# Re-enable the subscription
-$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
# Confirm that the conflict detection slot is created on Node A and the xmin
# value is valid.
@@ -280,7 +262,7 @@ 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;");
-($cmdret, $stdout, $stderr) = $node_A->psql(
+my ($cmdret, $stdout, $stderr) = $node_A->psql(
'postgres', qq(VACUUM (verbose) public.tab;)
);
--
2.31.1
v45-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v45-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From 1160e4901ff6fdcc85360490050fb655a9aea6f4 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v45] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 221 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 636 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 179 ++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2026 insertions(+), 186 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 126b8cfbad8..195967b28dd 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29566,7 +29566,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29610,7 +29612,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29640,6 +29644,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29662,8 +29668,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..df4a6176c10 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 +2389,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 b885513f765..b577e1dbb8d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 e5dbbe61b81..4d3de5bd979 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4ff246cd943..cebfb8b85c9 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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) < 19000)
+ 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 19"));
+
+ 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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..b69b47fd72c 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1182,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1204,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1256,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1277,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1318,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = MyReplicationSlot->data.xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..9db66579854 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,73 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher.
+ * Use FullTransactionId to prevent
+ * issues with transaction ID
+ * wraparound, where a new
+ * remote_oldestxid could falsely
+ * appear to originate from the past
+ * and block advancement */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher. Use
+ * FullTransactionId for consistency
+ * and to allow straightforward
+ * comparisons with remote_oldestxid. */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same
+ * reason as remote_nextxid */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +402,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +449,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3667,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3746,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3774,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3791,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3828,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3867,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3938,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3975,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;
@@ -3920,6 +4052,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5260,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5440,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5810,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index f9fec50ae88..2017d399961 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1937997ea67..19517c51a9b 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 fb063a2de42..9d826110a90 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -630,7 +630,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();
}
@@ -755,7 +755,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1944,48 +1944,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1998,12 +2030,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2115,6 +2159,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..9436fb7c310 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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 e46f02c6cc6..11e7e21478a 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..dbcf88be945 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ 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 8c2ea0b9587..bc4aba0526b 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2306,8 +2306,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3737,8 +3738,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_proc.dat b/src/include/catalog/pg_proc.dat
index d4650947c63..f0943fa9b0a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11793,6 +11793,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..58001aff466 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,179 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
v45-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v45-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From f7572087ccff4cf95f7c47f5989ba74f43d6ecbc Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 15:08:55 +0800
Subject: [PATCH v45 2/5] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 341 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4d3de5bd979..5221454ef67 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 b69b47fd72c..93e338147b7 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1184,6 +1191,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1216,7 +1224,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1229,7 +1238,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1278,14 +1287,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements: - Invalidate the
+ * slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it. - Drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain the conflict information. - if required,
+ * advance the slot's xmin to protect deleted tuples required for the
+ * conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1326,17 +1340,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1347,8 +1365,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1377,6 +1409,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1433,6 +1471,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1513,7 +1582,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;
@@ -1590,6 +1659,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 e593940da1e..514341175a5 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -457,6 +457,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3869,7 +3871,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4145,6 +4148,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4285,6 +4292,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4366,6 +4380,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4391,7 +4424,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4411,12 +4447,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4424,9 +4469,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4459,6 +4559,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 2017d399961..8f31e494914 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"},
};
/*
@@ -1668,6 +1669,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();
}
@@ -1786,6 +1792,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;
}
@@ -2056,6 +2068,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 511dc32d519..3a8ce6b74a8 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 f0943fa9b0a..3ed0b1be9a4 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 58001aff466..890bcdaac57 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -211,6 +211,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $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 the retain_conflict_info option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -246,6 +250,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$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');
+
###############################################################################
# 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.
@@ -255,7 +263,7 @@ ok( $node_A->poll_query_until(
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.31.1
v45-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v45-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From db64e6da7b7669687d59d294c450aaea69f56fa3 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v45 3/5] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 93e338147b7..0967fbd1fe5 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1389,6 +1389,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 514341175a5..5fff9f64a13 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -485,6 +485,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4148,10 +4150,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4431,6 +4429,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4438,6 +4455,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4479,9 +4497,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4510,19 +4527,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.31.1
v45-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v45-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 49c411e8a792c58d77f74542987417aa98ecf2d6 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:45:13 +0800
Subject: [PATCH v45 4/5] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 66 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 31 ++++-
15 files changed, 288 insertions(+), 46 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index c948263521e..16702bbc26b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5221454ef67..cac8d2f79d0 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 5fff9f64a13..ca60a668b1d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2818,17 +2818,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. */
@@ -3166,18 +3177,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;
}
@@ -4060,10 +4082,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4102,10 +4124,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4140,8 +4162,8 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4346,10 +4368,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop of
+ * required to detect update_delete reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
* maybe_advance_nonremovable_xid().
*
* XXX Consider waiting for the publisher's clock to catch up with the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 3ed0b1be9a4..220eaa4d20c 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 890bcdaac57..21407fb325b 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -257,6 +260,8 @@ is($result, qq(t), 'worker on node A 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.
+# And check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -270,6 +275,8 @@ is($result, qq(1|1
# 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;");
@@ -281,10 +288,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
--
2.31.1
On Tue, Jul 1, 2025 at 3:39 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tue, Jul 1, 2025 at 5:07 PM Dilip Kumar wrote:
On Tue, Jul 1, 2025 at 2:24 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:53 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, Jul 1, 2025 at 10:31 AM Dilip Kumar <dilipbalaut@gmail.com>
wrote:
On Mon, Jun 30, 2025 at 6:59 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Mon, Jun 30, 2025 at 7:22 PM Amit Kapila wrote:
I was looking at 0001, it mostly looks fine to me except this one
case. So here we need to ensure that commits must be acquired
after marking the flag, don't you think we need to ensure strict
statement ordering using memory barrier, or we think it's not
required and if so why?Good point. I also think we need a barrier here, but a write barrier
should be sufficient as we want ordering of two store operations.+1
RecordTransactionCommitPrepared() { .. + 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 transactions that have acquired commit + timestamp + * 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(); }I'm unsure whether the function call inherently acts as a memory
barrier, preventing the compiler from reordering these operations.
This needs to be confirmed.As per my understanding, function calls won't be a memory barrier. In
this regard, we need a similar change in RecordTransactionCommit as
well.Right, we need this in RecordTransactionCommit() as well.
Thanks for the comments! I also agree that the barrier is needed.
Here is V45 patch set.
I modified 0001, added write barriers, and improved some comments.
Thanks for working on this, I will have a look at it latest by tomorrow.
--
Regards,
Dilip Kumar
Google
On Tue, Jul 1, 2025 at 6:10 PM Zhijie Hou (Fujitsu) wrote:
Here is V45 patch set.
With the main patch set now stable, I am summarizing the performance tests
conducted before for reference.
In earlier tests [1]/messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com, we confirmed that in a pub-sub cluster with high workload
on the publisher (via pgbench), the patch had no impact on TPS (Transactions
Per Second) on the publisher. This indicates that the modifications to the
walsender responsible for replying to publisher status do not introduce
noticeable overhead.
Additionally, we confirmed that the patch, with its latest mechanism for
dynamically tuning the frequency of advancing slot.xmin, does not affect TPS on
the subscriber when minimal changes occur on the publisher. This test[2]/messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com
involved creating a pub-sub cluster and running pgbench on the subscriber to
monitor TPS. It further suggests that the logic for maintaining non-removable
xid in the apply worker does not introduce noticeable overhead for concurrent
user DMLs.
Furthermore, we tested running pgbench on both publisher and subscriber[3]/messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com.
Some regression was observed in TPS on the subscriber, because workload on the
publisher is pretty high and the apply workers must wait for the amount of
transactions with earlier timestamps to be applied and flushed before advancing
the non-removable XID to remove dead tuples. This is the expected behavior of
this approach since the patch's main goal is to retain dead tuples for reliable
conflict detection.
When discussing the regression, we considered providing a workaround for users
to recover from the regression (the 0002 of the latest patch set). We
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 apply worker cannot catch
up with the publisher's workload. In short, the conflict detection replication slot
will be invalidated if lag time exceeds the specified GUC value.
In performance tests[4]/messages/by-id/OSCPR01MB14966F39BE1732B9E433023BFF5E72@OSCPR01MB14966.jpnprd01.prod.outlook.com, we confirmed that the slot would be invalidated as
expected when the workload on the publisher was high, and it would not get
invalidated anymore after reducing the workload. This shows even if the slot
has been invalidated once, users can continue to detect the update_deleted
conflict by reduce the workload on the publisher.
The design of the patch set was not changed since the last performance test;
only some code enhancements have been made. Therefore, I think the results and
findings from the previous performance tests are still valid. However, if
necessary, we can rerun all the tests on the latest patch set to verify the
same.
[1]: /messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com
[2]: /messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com
[3]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
[4]: /messages/by-id/OSCPR01MB14966F39BE1732B9E433023BFF5E72@OSCPR01MB14966.jpnprd01.prod.outlook.com
Best Regards,
Hou zj
On Wed, Jul 2, 2025 at 2:03 PM Zhijie Hou (Fujitsu) wrote:
On Tue, Jul 1, 2025 at 6:10 PM Zhijie Hou (Fujitsu) wrote:
Here is V45 patch set.
With the main patch set now stable, I am summarizing the performance tests
conducted before for reference.In earlier tests [1], we confirmed that in a pub-sub cluster with high workload
on the publisher (via pgbench), the patch had no impact on TPS (Transactions
Per Second) on the publisher. This indicates that the modifications to the
walsender responsible for replying to publisher status do not introduce
noticeable overhead.Additionally, we confirmed that the patch, with its latest mechanism for
dynamically tuning the frequency of advancing slot.xmin, does not affect TPS
on the subscriber when minimal changes occur on the publisher. This test[2]
involved creating a pub-sub cluster and running pgbench on the subscriber to
monitor TPS. It further suggests that the logic for maintaining non-removable
xid in the apply worker does not introduce noticeable overhead for concurrent
user DMLs.Furthermore, we tested running pgbench on both publisher and subscriber[3].
Some regression was observed in TPS on the subscriber, because workload on
the publisher is pretty high and the apply workers must wait for the amount of
transactions with earlier timestamps to be applied and flushed before
advancing the non-removable XID to remove dead tuples. This is the expected
behavior of this approach since the patch's main goal is to retain dead tuples
for reliable conflict detection.When discussing the regression, we considered providing a workaround for
users to recover from the regression (the 0002 of the latest patch set). We
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 apply worker cannot catch up
with the publisher's workload. In short, the conflict detection replication slot
will be invalidated if lag time exceeds the specified GUC value.In performance tests[4], we confirmed that the slot would be invalidated as
expected when the workload on the publisher was high, and it would not get
invalidated anymore after reducing the workload. This shows even if the slot
has been invalidated once, users can continue to detect the update_deleted
conflict by reduce the workload on the publisher.The design of the patch set was not changed since the last performance test;
only some code enhancements have been made. Therefore, I think the results
and findings from the previous performance tests are still valid. However, if
necessary, we can rerun all the tests on the latest patch set to verify the same.
During local testing, I discovered a bug caused by my oversight in assigning
the new xmin to slot.effective, which resulted in dead tuples remaining
non-removable until restart. I apologize for the error and have provided
corrected patches. Kindly use the latest patch set for performance testing.
Best Regards,
Hou zj
Attachments:
v46-0005-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v46-0005-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From 50715c5903fd77add03153e3aa56ec89ea2b06ec Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 2 Jul 2025 15:21:35 +0800
Subject: [PATCH v46 5/5] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 70 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 +++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 101 +++++++-----------
src/include/replication/logicalworker.h | 3 +-
src/include/replication/worker_internal.h | 29 ++++-
src/test/subscription/t/035_conflicts.pl | 26 +----
9 files changed, 132 insertions(+), 194 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index cebfb8b85c9..ff226063458 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -649,7 +649,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
- !opts.enabled, WARNING);
+ !opts.enabled);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -1065,22 +1065,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on
- * the publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1092,41 +1084,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option
- * when the subscription is enabled to prevent race conditions arising
- * from the new option value being acknowledged asynchronously by the
- * launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1396,28 +1353,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the
- * subscription has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker
- * processes have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
CheckSubConflictInfoRetention(opts.retainconflictinfo,
- true, !sub->enabled, NOTICE);
+ true, !sub->enabled);
/*
* Notify the launcher to manage the replication slot for
@@ -1458,8 +1399,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* comments atop CheckSubConflictInfoRetention() for details.
*/
CheckSubConflictInfoRetention(sub->retainconflictinfo,
- opts.enabled, !opts.enabled,
- WARNING);
+ opts.enabled, !opts.enabled);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ca96f219405..5133854c9bc 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1192,7 +1186,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1265,8 +1259,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1346,7 +1339,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1355,13 +1348,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1381,10 +1368,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1449,19 +1440,48 @@ acquire_conflict_slot_if_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = new_xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index dae5100d343..32f1bb6a2c4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -313,8 +313,8 @@ typedef struct RetainConflictInfoData
* (RCI_WAIT_FOR_LOCAL_FLUSH). Use
* FullTransactionId for the same
* reason as remote_nextxid */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -487,8 +487,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4210,6 +4208,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4237,17 +4237,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4255,7 +4258,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4364,7 +4367,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4456,22 +4459,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4484,7 +4481,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4510,7 +4507,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4529,7 +4526,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4564,7 +4561,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4721,6 +4718,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 was changed. 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 ||
@@ -4730,7 +4736,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,
@@ -5410,30 +5417,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5596,7 +5579,7 @@ DisableSubscriptionAndExit(void)
* context.
*/
CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
- true, WARNING);
+ true);
proc_exit(0);
}
@@ -5974,18 +5957,13 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_conflict_info for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING if the subscription is disabled. Do not raise an ERROR since
+ * as long as the subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
- bool sub_disabled, int elevel_for_sub_disabled)
+ bool sub_disabled)
{
- Assert(elevel_for_sub_disabled == NOTICE ||
- elevel_for_sub_disabled == WARNING);
-
if (!retain_conflict_info)
return;
@@ -6003,10 +5981,9 @@ CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
"track_commit_timestamp"));
if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_conflict_info") : 0);
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
}
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index ebbb3ef09c7..2d006e7888e 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -32,7 +32,6 @@ extern void AtEOXact_LogicalRepWorkers(bool isCommit);
extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
bool check_guc,
- bool sub_disabled,
- int elevel_for_sub_disabled);
+ bool sub_disabled);
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 21407fb325b..bca7ec90300 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -219,31 +219,13 @@ my $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Check that the retain_conflict_info option can be enabled only for disabled
-# subscriptions. Validate the NOTICE message during the subscription DDL, and
-# ensure the conflict detection slot is created upon enabling the
+# Check that the conflict detection slot is created after enabling
# retain_conflict_info option.
##################################################
-# Alter retain_conflict_info for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
-# Disable the subscription
-$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
-
-# Enable retain_conflict_info for disabled subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+# Enable retain_conflict_info for the subscription
+$node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
-ok( $stderr =~
- /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
- "altering retain_conflict_info is allowed for disabled subscription");
-
-# Re-enable the subscription
-$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
# Confirm that the conflict detection slot is created on Node A and the xmin
# value is valid.
@@ -280,7 +262,7 @@ 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;");
-($cmdret, $stdout, $stderr) = $node_A->psql(
+my ($cmdret, $stdout, $stderr) = $node_A->psql(
'postgres', qq(VACUUM (verbose) public.tab;)
);
--
2.30.0.windows.2
v46-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v46-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From c35dfd29746352f753e3e81798a02477013593e9 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v46 1/5] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 227 ++++++-
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 221 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 636 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 179 ++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2026 insertions(+), 186 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 4f9192316e0..de9617e2893 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 810b2b50f0d..1d593079fd9 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29592,7 +29592,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29636,7 +29638,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29666,6 +29670,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29688,8 +29694,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..ed835032d27 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,45 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..df4a6176c10 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 +2389,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 b885513f765..b577e1dbb8d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 e5dbbe61b81..4d3de5bd979 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4ff246cd943..cebfb8b85c9 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_pub_conflict_info_retention(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);
@@ -210,7 +217,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)
@@ -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);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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 +747,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
check_publications_origin(wrconn, publications, opts.copy_data,
opts.origin, NULL, 0, stmt->subname);
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
+
/*
* Set sync state based on if we were asked to do data copy or
* not.
@@ -1040,18 +1065,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1092,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,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 +1230,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 +1390,46 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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] =
@@ -1347,6 +1452,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1469,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 +1491,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 +1697,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 +1714,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 +1730,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_pub_conflict_info_retention(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 +2333,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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) < 19000)
+ 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 19"));
+
+ 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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..dfbe268f959 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1182,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1204,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1256,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1277,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1318,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7b4e8629553..5febd154b6b 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..9db66579854 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,73 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher.
+ * Use FullTransactionId to prevent
+ * issues with transaction ID
+ * wraparound, where a new
+ * remote_oldestxid could falsely
+ * appear to originate from the past
+ * and block advancement */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher. Use
+ * FullTransactionId for consistency
+ * and to allow straightforward
+ * comparisons with remote_oldestxid. */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same
+ * reason as remote_nextxid */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +402,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +449,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3667,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3746,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3774,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3791,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3828,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3867,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3938,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3975,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;
@@ -3920,6 +4052,426 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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 seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5260,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5440,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5810,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index f9fec50ae88..2017d399961 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1937997ea67..19517c51a9b 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 fb063a2de42..9d826110a90 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -630,7 +630,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();
}
@@ -755,7 +755,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1944,48 +1944,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1998,12 +2030,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2115,6 +2159,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..9436fb7c310 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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 e46f02c6cc6..11e7e21478a 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..dbcf88be945 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ 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 8c2ea0b9587..bc4aba0526b 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2306,8 +2306,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3737,8 +3738,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_proc.dat b/src/include/catalog/pg_proc.dat
index d4650947c63..f0943fa9b0a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11793,6 +11793,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..58001aff466 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,179 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 220e5a4f6b3..281c6ecc9cf 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
v46-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v46-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 568352683462caa38ff81513a08f51b1cc3d8f5a Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 15:08:55 +0800
Subject: [PATCH v46 2/5] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 341 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 986ae1f543d..1d3ee5a9ed7 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
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4d3de5bd979..5221454ef67 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 dfbe268f959..ededd8d65a1 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1184,6 +1191,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1216,7 +1224,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1229,7 +1238,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1278,14 +1287,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements: - Invalidate the
+ * slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it. - Drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain the conflict information. - if required,
+ * advance the slot's xmin to protect deleted tuples required for the
+ * conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1326,17 +1340,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1347,8 +1365,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1377,6 +1409,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1433,6 +1471,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1513,7 +1582,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;
@@ -1590,6 +1659,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 9db66579854..00cfedf3663 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -459,6 +459,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3871,7 +3873,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4147,6 +4150,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4287,6 +4294,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4368,6 +4382,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4393,7 +4426,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4413,12 +4449,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4426,9 +4471,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4461,6 +4561,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 2017d399961..8f31e494914 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"},
};
/*
@@ -1668,6 +1669,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();
}
@@ -1786,6 +1792,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;
}
@@ -2056,6 +2068,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 511dc32d519..3a8ce6b74a8 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 f0943fa9b0a..3ed0b1be9a4 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 58001aff466..890bcdaac57 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -211,6 +211,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $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 the retain_conflict_info option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -246,6 +250,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$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');
+
###############################################################################
# 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.
@@ -255,7 +263,7 @@ ok( $node_A->poll_query_until(
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.30.0.windows.2
v46-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v46-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From b151421b8079763f4251f9d1ffc869fa0556b5a3 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v46 3/5] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ededd8d65a1..ca96f219405 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1389,6 +1389,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 00cfedf3663..f3ae5f81167 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -487,6 +487,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4150,10 +4152,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4433,6 +4431,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4440,6 +4457,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4481,9 +4499,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4512,19 +4529,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
v46-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v46-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From f6c8d176c71dbf59b5c2c1ee700e06beb882af9f Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:45:13 +0800
Subject: [PATCH v46 4/5] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 66 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 31 ++++-
15 files changed, 288 insertions(+), 46 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index de9617e2893..d835ec8eb9f 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index ed835032d27..cd4aa528577 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5221454ef67..cac8d2f79d0 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 f3ae5f81167..dae5100d343 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2820,17 +2820,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. */
@@ -3168,18 +3179,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;
}
@@ -4062,10 +4084,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4104,10 +4126,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4142,8 +4164,8 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4348,10 +4370,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop of
+ * required to detect update_delete reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
* maybe_advance_nonremovable_xid().
*
* XXX Consider waiting for the publisher's clock to catch up with the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 3ed0b1be9a4..220eaa4d20c 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 890bcdaac57..21407fb325b 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -257,6 +260,8 @@ is($result, qq(t), 'worker on node A 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.
+# And check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -270,6 +275,8 @@ is($result, qq(1|1
# 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;");
@@ -281,10 +288,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
--
2.30.0.windows.2
On Wed, Jul 2, 2025 at 12:58 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
During local testing, I discovered a bug caused by my oversight in assigning
the new xmin to slot.effective, which resulted in dead tuples remaining
non-removable until restart. I apologize for the error and have provided
corrected patches. Kindly use the latest patch set for performance testing.
You changes related to write barrier LGTM, however I have question
regarding below change, IIUC, in logical replication
MyReplicationSlot->effective_xmin should be the xmin value which has
been flushed to the disk, but here we are just setting "data.xmin =
new_xmin;" and marking the slot dirty so I believe its not been yet
flushed to the disk right?
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
..
}
--
Regards,
Dilip Kumar
Google
On Thu, Jul 3, 2025 at 10:26 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Jul 2, 2025 at 12:58 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:During local testing, I discovered a bug caused by my oversight in assigning
the new xmin to slot.effective, which resulted in dead tuples remaining
non-removable until restart. I apologize for the error and have provided
corrected patches. Kindly use the latest patch set for performance testing.You changes related to write barrier LGTM, however I have question
regarding below change, IIUC, in logical replication
MyReplicationSlot->effective_xmin should be the xmin value which has
been flushed to the disk, but here we are just setting "data.xmin =
new_xmin;" and marking the slot dirty so I believe its not been yet
flushed to the disk right?
Yes, because this is a physical slot and we need to follow
PhysicalConfirmReceivedLocation()/PhysicalReplicationSlotNewXmin().
The patch has kept a comment in advance_conflict_slot_xmin() as to why
it is okay not to flush the slot immediately.
--
With Regards,
Amit Kapila.
On Thu, Jul 3, 2025 at 10:43 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 3, 2025 at 10:26 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Jul 2, 2025 at 12:58 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:During local testing, I discovered a bug caused by my oversight in assigning
the new xmin to slot.effective, which resulted in dead tuples remaining
non-removable until restart. I apologize for the error and have provided
corrected patches. Kindly use the latest patch set for performance testing.You changes related to write barrier LGTM, however I have question
regarding below change, IIUC, in logical replication
MyReplicationSlot->effective_xmin should be the xmin value which has
been flushed to the disk, but here we are just setting "data.xmin =
new_xmin;" and marking the slot dirty so I believe its not been yet
flushed to the disk right?Yes, because this is a physical slot and we need to follow
PhysicalConfirmReceivedLocation()/PhysicalReplicationSlotNewXmin().
The patch has kept a comment in advance_conflict_slot_xmin() as to why
it is okay not to flush the slot immediately.
Oh right, I forgot its physical slot. I think we are good, thanks.
--
Regards,
Dilip Kumar
Google
On Thu, Jul 3, 2025 at 10:57 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Thu, Jul 3, 2025 at 10:43 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 3, 2025 at 10:26 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
You changes related to write barrier LGTM, however I have question
regarding below change, IIUC, in logical replication
MyReplicationSlot->effective_xmin should be the xmin value which has
been flushed to the disk, but here we are just setting "data.xmin =
new_xmin;" and marking the slot dirty so I believe its not been yet
flushed to the disk right?Yes, because this is a physical slot and we need to follow
PhysicalConfirmReceivedLocation()/PhysicalReplicationSlotNewXmin().
The patch has kept a comment in advance_conflict_slot_xmin() as to why
it is okay not to flush the slot immediately.Oh right, I forgot its physical slot. I think we are good, thanks.
BTW, I wanted to clarify one more point related to this part of the
patch. One difference between PhysicalReplicationSlotNewXmin() and
advance_conflict_slot_xmin() is that the former updates both
catalog_xmin and xmin for the slot, but later updates only the slot's
xmin. Can we see any reason to update both in our case? For example,
there is one case where the caller of
ProcArrayGetReplicationSlotXmin() expects that the catalog_xmin must
be set when required, but as far as I can see, it is required only
when logical slots are present, so we should be okay with that case.
The other case to consider is vacuum calling
GetOldestNonRemovableTransactionId() to get the cutoff xid to remove
deleted rows. This returns the xmin horizon based on the type of table
(user table, catalog table, etc.). Now, in this case,
ComputeXidHorizons() will first set data xmin for
catalog_oldest_nonremovable xid, and then if slot_catalog_xmin is
smaller, it uses that value. So, for this computation as well, setting
just slot's xmin in advance_conflict_slot_xmin() should be sufficient,
as we will anyway set both xmin and catalog_xmin to the same values.
By this theory, it doesn't matter whether we set catalog_xmin for
physical slots or not till we are setting the slot's xmin. IIUC,
catalog_xmin is required to be set for logical slots because during
logical decoding, we access only catalog tables, so we need to protect
those, and the catalog_xmin value is used for that.
Thoughts?
--
With Regards,
Amit Kapila.
On Thu, Jul 3, 2025 at 4:21 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 3, 2025 at 10:57 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Thu, Jul 3, 2025 at 10:43 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 3, 2025 at 10:26 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
You changes related to write barrier LGTM, however I have question
regarding below change, IIUC, in logical replication
MyReplicationSlot->effective_xmin should be the xmin value which has
been flushed to the disk, but here we are just setting "data.xmin =
new_xmin;" and marking the slot dirty so I believe its not been yet
flushed to the disk right?Yes, because this is a physical slot and we need to follow
PhysicalConfirmReceivedLocation()/PhysicalReplicationSlotNewXmin().
The patch has kept a comment in advance_conflict_slot_xmin() as to why
it is okay not to flush the slot immediately.Oh right, I forgot its physical slot. I think we are good, thanks.
BTW, I wanted to clarify one more point related to this part of the
patch. One difference between PhysicalReplicationSlotNewXmin() and
advance_conflict_slot_xmin() is that the former updates both
catalog_xmin and xmin for the slot, but later updates only the slot's
xmin. Can we see any reason to update both in our case?
IMHO the purpose of these 2 functions are different, I think the
PhysicalReplicationSlotNewXmin() update the xmin in response to
hot_standby_feedback and the purpose of that is to avoid removing
anything on primary until it is no longer required by the standby so
that we do not create conflict or query cancellation. So it has to
consider the data required by active queries, physical/logical
replication slots on standby etc. Whereas the purpose of
advance_conflict_slot_xmin() to prevent the tuple from removing on the
subscriber which might required for the conflict detection on this
node, the other aspect of not removing the tuple which is required for
the logical/physical replication slots on this node is already taken
care by other slots. So I think this is a very specific purpose build
slot which just has one specific task, so I feel we are good with what
we have.
--
Regards,
Dilip Kumar
Google
On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical replication
setups, such as (A -> B -> C). When retain_conflict_info is enabled on Node C,
it may not retain information necessary for conflict detection when applying
changes originally replicated from Node A. This happens because Node C only
waits for locally originated changes on Node B to be applied before advancing
the non-removable transaction ID.
For example, Consider a logical replication setup as mentioned above : A -> B -> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.
Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)
- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no concurrent
transactions from Node B. It is unaware of Node A’s concurrent update.
@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)
@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may already have
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.
Note that, as per decoding logic Node C sees the commit timestamp of the update
as 9:00 (origin commit_ts from Node A), not 9:04 (commit time on Node B). In
this case, since the UPDATE's timestamp is earlier than the DELETE, Node C
should ideally detect an update_deleted conflict. However, it cannot, because
it no longer retains the deleted tuple.
Even if Node C attempts to retrieve the latest WAL position from Node A, Node C
doesn't maintain any LSN which we could use to compare with it.
This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a physical
standby, as the required transaction information from the original primary is
unavailable. Moreover, this limitation is relevant only when the subscription
origin option is set to ANY, as only in that case changes from other origins
can be replicated. Since retain_conflict_info is primarily useful for conflict
detection in bidirectional clusters where the origin option is set to NONE,
this limitation appears acceptable.
Given these findings, to help users avoid unintended configurations, we plan to
issue a warning in scenarios where replicated changes may include origins other
than the direct publisher, similar to the existing checks in the
check_publications_origin() function.
Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.
A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.
Best Regards,
Hou zj
Attachments:
v47-0004-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v47-0004-Support-the-conflict-detection-for-update_delete.patchDownload
From 5470461858429fb2bd94b215e2047b3cd43ce6cc Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Thu, 26 Jun 2025 10:45:13 +0800
Subject: [PATCH v47 4/5] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 66 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 31 ++++-
15 files changed, 288 insertions(+), 46 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index de9617e2893..d835ec8eb9f 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretainconflictinfo</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index 2394f282253..34133395864 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1743,6 +1743,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 84e5a48181c..dbe8ee6db16 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2212,6 +2212,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 7782246727e..fce4500280f 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 5221454ef67..cac8d2f79d0 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1390,6 +1390,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 53ddd25c42d..ce8eec5ca20 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 8bf21938698..900ff7448fe 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2820,17 +2820,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. */
@@ -3168,18 +3179,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;
}
@@ -4062,10 +4084,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
*
* 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 conflicts reliably. This helps to retain the required commit_ts
- * module information, which further helps to detect update_origin_differs and
- * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
- * remove the required information.
+ * to detect update_deleted conflicts. Additionally, this helps to retain
+ * the required commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
*
* The non-removable transaction ID is advanced to the oldest running
* transaction ID once all concurrent transactions on the publisher have been
@@ -4104,10 +4126,10 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -4159,8 +4181,8 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4365,10 +4387,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop of
+ * required to detect update_delete reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop of
* maybe_advance_nonremovable_xid().
*
* XXX Consider waiting for the publisher's clock to catch up with the
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 3ed0b1be9a4..220eaa4d20c 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 436fe445d64..243164aada8 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 79ed5233edb..d0428ac47fb 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 f06fb77ad8d..6c9c90c04d8 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -273,6 +276,8 @@ $node_A->psql('postgres',
###############################################################################
# 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
@@ -286,6 +291,8 @@ is($result, qq(1|1
# 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;");
@@ -297,10 +304,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
--
2.30.0.windows.2
v47-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v47-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From 77beeae876688897d6dd591b080e571875c446ac Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v47] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 50 +-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 358 ++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 221 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 653 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 +++--
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 195 +++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2168 insertions(+), 215 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index fa86c569dc4..c948263521e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 126b8cfbad8..195967b28dd 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29566,7 +29566,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29610,7 +29612,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29640,6 +29644,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29662,8 +29668,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..7782246727e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,52 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+
+ <para>
+ Enabling this option ensures retention of information useful for
+ conflict detection solely for changes occurring locally on the
+ publisher. For the changes originating from different origins,
+ reliable conflict detection cannot be guaranteed.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 73a80559194..df4a6176c10 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 in logical replication. See
+ * GetOldestActiveTransactionId(true) 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,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 +2389,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 b885513f765..b577e1dbb8d 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 6ce979f2d8b..2dcda37bc77 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 e5dbbe61b81..4d3de5bd979 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4ff246cd943..20bb26d5c49 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;
@@ -105,8 +108,11 @@ typedef struct SubOpts
static List *fetch_table_list(WalReceiverConn *wrconn, List *publications);
static void check_publications_origin(WalReceiverConn *wrconn,
List *publications, bool copydata,
- char *origin, Oid *subrel_local_oids,
- int subrel_count, char *subname);
+ bool retain_conflict_info, char *origin,
+ Oid *subrel_local_oids, int subrel_count,
+ char *subname);
+static void check_pub_conflict_info_retention(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 +168,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);
@@ -210,7 +218,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)
@@ -307,6 +315,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 +580,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);
/*
@@ -630,6 +648,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +692,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)
@@ -722,7 +746,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
{
check_publications(wrconn, publications);
check_publications_origin(wrconn, publications, opts.copy_data,
- opts.origin, NULL, 0, stmt->subname);
+ opts.retainconflictinfo, opts.origin,
+ NULL, 0, stmt->subname);
+
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
/*
* Set sync state based on if we were asked to do data copy or
@@ -881,8 +908,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
sizeof(Oid), oid_cmp);
check_publications_origin(wrconn, sub->publications, copy_data,
- sub->origin, subrel_local_oids,
- subrel_count, sub->name);
+ sub->retainconflictinfo, sub->origin,
+ subrel_local_oids, subrel_count, sub->name);
/*
* Rels that we want to remove from subscription and drop any slots
@@ -1040,18 +1067,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1094,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1176,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_tuple = false;
bool update_failover = false;
bool update_two_phase = false;
+ bool check_pub_rci = false;
+ bool retain_conflict_info;
+ char *origin;
Subscription *sub;
Form_pg_subscription form;
bits32 supported_opts;
@@ -1137,6 +1206,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
sub = GetSubscription(subid, false);
+ retain_conflict_info = sub->retainconflictinfo;
+ origin = sub->origin;
+
/*
* Don't allow non-superuser modification of a subscription with
* password_required=false.
@@ -1165,7 +1237,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,11 +1397,62 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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();
+
+ check_pub_rci = opts.retainconflictinfo;
+ retain_conflict_info = opts.retainconflictinfo;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
CStringGetTextDatum(opts.origin);
replaces[Anum_pg_subscription_suborigin - 1] = true;
+
+ /*
+ * Check if changes from different origins may be received
+ * from the publisher when the origin is changed to ANY
+ * and retain_conflict_info is enabled.
+ */
+ check_pub_rci = retain_conflict_info &&
+ pg_strcasecmp(opts.origin, LOGICALREP_ORIGIN_ANY) == 0;
+
+ origin = opts.origin;
}
update_tuple = true;
@@ -1347,6 +1470,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1487,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.
+ */
+ check_pub_rci = sub->retainconflictinfo && opts.enabled;
break;
}
@@ -1369,6 +1509,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.
+ */
+ check_pub_rci = sub->retainconflictinfo;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1715,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 || check_pub_rci)
{
bool must_use_password;
char *err;
@@ -1584,10 +1732,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 +1748,16 @@ 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_pub_conflict_info_retention(wrconn, retain_conflict_info);
+
+ check_publications_origin(wrconn, sub->publications, false,
+ retain_conflict_info, origin, NULL, 0,
+ sub->name);
+
+ 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();
{
@@ -2086,20 +2245,29 @@ AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId)
* Check and log a warning if the publisher has subscribed to the same table,
* its partition ancestors (if it's a partition), or its partition children (if
* it's a partitioned table), from some other publishers. This check is
- * required only if "copy_data = true" and "origin = none" for CREATE
- * SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements to notify the
- * user that data having origin might have been copied.
+ * required in the following scenarios:
*
- * This check need not be performed on the tables that are already added
- * because incremental sync for those tables will happen through WAL and the
- * origin of the data can be identified from the WAL records.
+ * 1) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "copy_data = true" and "origin = none":
+ * - Warn the user that data with an origin might have been copied.
+ * - This check is skipped for tables already added, as incremental sync via
+ * WAL allows origin tracking. The list of such tables is in
+ * subrel_local_oids.
*
- * subrel_local_oids contains the list of relation oids that are already
- * present on the subscriber.
+ * 2) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "retain_conflict_info = true" and "origin = any", and for ALTER
+ * SUBSCRIPTION statements that modify retain_conflict_info or origin, or
+ * when the publisher's status changes (e.g., due to a connection string
+ * update):
+ * - Warn the user that only conflict detection info for local changes on
+ * the publisher is retained. Data from other origins may lack sufficient
+ * details for reliable conflict detection.
+ * - See comments in maybe_advance_nonremovable_xid() for more details.
*/
static void
check_publications_origin(WalReceiverConn *wrconn, List *publications,
- bool copydata, char *origin, Oid *subrel_local_oids,
+ bool copydata, bool retain_conflict_info,
+ char *origin, Oid *subrel_local_oids,
int subrel_count, char *subname)
{
WalRcvExecResult *res;
@@ -2108,9 +2276,29 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
Oid tableRow[1] = {TEXTOID};
List *publist = NIL;
int i;
+ bool check_rci;
+ bool check_table_sync;
+ bool origin_none = origin &&
+ pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) == 0;
+
+ /*
+ * Enable retain_conflict_info checks only when origin is set to 'any',
+ * since with origin='none' only local changes are replicated to the
+ * subscriber.
+ */
+ check_rci = retain_conflict_info && !origin_none;
- if (!copydata || !origin ||
- (pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) != 0))
+ /*
+ * Enable table synchronization checks only when origin is 'none', to
+ * ensure that data from other origins is not inadvertently copied.
+ */
+ check_table_sync = copydata && origin_none;
+
+ /* retain_conflict_info and table sync checks occur separately */
+ Assert(!(check_rci && check_table_sync));
+
+ /* Return if no checks are required */
+ if (!check_rci && !check_table_sync)
return;
initStringInfo(&cmd);
@@ -2129,16 +2317,23 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
/*
* In case of ALTER SUBSCRIPTION ... REFRESH, subrel_local_oids contains
* the list of relation oids that are already present on the subscriber.
- * This check should be skipped for these tables.
+ * This check should be skipped for these tables if checking for table
+ * sync scenario. However, when handling the retain_conflict_info
+ * scenario, ensure all tables are checked, as some existing tables may
+ * now include changes from other origins due to newly created
+ * subscriptions on the publisher.
*/
- for (i = 0; i < subrel_count; i++)
+ if (check_table_sync)
{
- Oid relid = subrel_local_oids[i];
- char *schemaname = get_namespace_name(get_rel_namespace(relid));
- char *tablename = get_rel_name(relid);
+ for (i = 0; i < subrel_count; i++)
+ {
+ Oid relid = subrel_local_oids[i];
+ char *schemaname = get_namespace_name(get_rel_namespace(relid));
+ char *tablename = get_rel_name(relid);
- appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
- schemaname, tablename);
+ appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
+ schemaname, tablename);
+ }
}
res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
@@ -2173,22 +2368,37 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
* XXX: For simplicity, we don't check whether the table has any data or
* not. If the table doesn't have any data then we don't need to
* distinguish between data having origin and data not having origin so we
- * can avoid logging a warning in that case.
+ * can avoid logging a warning for table sync scenario.
*/
if (publist)
{
StringInfo pubnames = makeStringInfo();
+ StringInfo err_msg = makeStringInfo();
+ StringInfo err_hint = makeStringInfo();
/* Prepare the list of publication(s) for warning message. */
GetPublicationsStr(publist, pubnames, false);
+
+ if (check_table_sync)
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin"),
+ subname);
+ appendStringInfoString(err_hint, _("Verify that initial data copied from the publisher tables did not come from other origins."));
+ }
+ else
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins"),
+ subname);
+ appendStringInfoString(err_hint, _("Consider using origin = NONE or disabling retain_conflict_info."));
+ }
+
ereport(WARNING,
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin",
- subname),
- errdetail_plural("The subscription being created subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
- "The subscription being created subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
+ errmsg_internal("%s", err_msg->data),
+ errdetail_plural("The subscription subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
+ "The subscription subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
list_length(publist), pubnames->data),
- errhint("Verify that initial data copied from the publisher tables did not come from other origins."));
+ errhint_internal("%s", err_hint->data));
}
ExecDropSingleTupleTableSlot(slot);
@@ -2196,6 +2406,58 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop maybe_advance_nonremovable_xid() for a detailed
+ * explanation.
+ */
+static void
+check_pub_conflict_info_retention(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) < 19000)
+ 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 19"));
+
+ 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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..dfbe268f959 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1182,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1204,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1256,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1277,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1318,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index c4299c76fb1..bb2f5671735 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..f83193b7781 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -140,6 +140,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +167,14 @@
#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"
#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"
@@ -268,6 +271,73 @@ 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 */
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher.
+ * Use FullTransactionId to prevent
+ * issues with transaction ID
+ * wraparound, where a new
+ * remote_oldestxid could falsely
+ * appear to originate from the past
+ * and block advancement */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher. Use
+ * FullTransactionId for consistency
+ * and to allow straightforward
+ * comparisons with remote_oldestxid. */
+ TimestampTz reply_time; /* when the publisher responds with status */
+ FullTransactionId remote_wait_for; /* publisher transaction ID that must
+ * be awaited to complete before
+ * entering the final phase
+ * (RCI_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same
+ * reason as remote_nextxid */
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +402,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +449,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3667,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3746,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3774,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3791,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3828,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3867,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +3938,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +3975,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;
@@ -3920,6 +4052,443 @@ 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 conflicts reliably. This helps to retain the required commit_ts
+ * module information, which further helps to detect update_origin_differs and
+ * delete_origin_differs conflicts reliably, as otherwise, vacuum freeze could
+ * remove the required information.
+ *
+ * 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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.
+ *
+ * Similarly, when the publisher has subscribed to another publisher,
+ * information necessary for conflict detection cannot be retained for
+ * changes from origins other than the publisher. This is because the publisher
+ * lacks the transaction status of other publishers it subscribes to.
+ * Consequently, the non-removable transaction ID might be advanced prematurely
+ * before changes from other origins have been fully applied.
+ *
+ * XXX Retaining information for changes from other origins might be possible
+ * by requesting the subscription on that origin to enable retain_conflict_info
+ * and fetching the conflict detection slot.xmin along with the publisher's
+ * status. In the RCI_WAIT_FOR_PUBLISHER_STATUS phase, the apply worker could
+ * wait for the remote slot's xmin to reach the oldest active transaction ID,
+ * ensuring that all transactions from other origins have been applied on the
+ * publisher, thereby getting the latest WAL position that includes all
+ * concurrent changes. However, this approach may impact performance, so it
+ * might not worth the effort.
+ *
+ * XXX It seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop of maybe_advance_nonremovable_xid() for
+ * details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop of
+ * maybe_advance_nonremovable_xid().
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5277,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5457,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5827,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index f9fec50ae88..2017d399961 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1937997ea67..19517c51a9b 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 fb063a2de42..9d826110a90 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -27,7 +27,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -630,7 +630,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();
}
@@ -755,7 +755,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -1944,48 +1944,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -1998,12 +2030,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2115,6 +2159,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..9436fb7c310 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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 e46f02c6cc6..11e7e21478a 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..dbcf88be945 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ 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 8c2ea0b9587..bc4aba0526b 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2306,8 +2306,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3737,8 +3738,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_proc.dat b/src/include/catalog/pg_proc.dat
index d4650947c63..f0943fa9b0a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11793,6 +11793,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..9b675b95407 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,195 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+##################################################
+# Check the WARNING when changing the origin to ANY, if retain_conflict_info is
+# enabled. This warns of the possibility of receiving changes from origins
+# other than the publisher.
+##################################################
+
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = any);");
+ok( $stderr =~
+ /WARNING: subscription "tap_sub_a_b" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins/,
+ "warn of the possibility of receiving changes from origins other than the publisher");
+
+# Reset the origin to none
+$node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = none);");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 32d6e718adc..2b12a1ee919 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2559,6 +2559,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
v47-0005-Allow-altering-retain_conflict_info-for-enabled-.patchapplication/octet-stream; name=v47-0005-Allow-altering-retain_conflict_info-for-enabled-.patchDownload
From f04716451b0ff5f0b340aad76297e422464aaa80 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 2 Jul 2025 15:21:35 +0800
Subject: [PATCH v47 5/5] Allow altering retain_conflict_info for enabled
subscription
This patch removes the restriction on altering retain_conflict_info when the
subscription is enabled, and resolves race condition issues caused by the new
option value being asynchronously acknowledged by the launcher and apply
workers.
First, without this restriction, the oldest_nonremovable_xid maintained by the
apply worker could become invalid during transaction ID wraparound if the slot
does not yet exist but the worker has already started to maintain the value.
For example, this race condition can occur when a user disables and immediately
re-enables the retain_conflict_info option. In this case, the launcher might
drop the slot upon noticing the disable action, while the apply worker may keep
maintaining oldest_nonremovable_xid without noticing the option change. During
this period, a transaction ID wraparound could falsely make this ID appear as
if it originates from the future w.r.t the transaction ID stored in the slot
maintained by launcher. To address this issue, we define the
oldest_nonremovable_xid as FullTransactionID so that even if the warparound
happens, we can correctly identity if the transaction ID a old or new one.
Second, when the launcher collects a new xmin value for the conflict detection
replication slot, the new xmin might be older than the current xmin. This can
happen when: 1) retain_conflict_info is disabled and immediately re-enabled
similar to the case mentioned in the first issue. In this case, if the some
transaction IDs are assigned before the slot creation, the newly created slot
could have a newer xmin compared to the worker. 2) the user enables
retain_conflict_info concurrently with the launcher starting the worker, the
apply worker may start calculating oldest_nonremovable_xid before the launcher
notices the enable action. Consequently, the launcher may update slot.xmin to a
newer value than that maintained by the worker. In subsequent cycles, upon
integrating the worker's oldest_nonremovable_xid, the launcher might detect a
regression in the calculated xmin, necessitating additional handling. We
address this by adding a safeguard check when advancing slot.xmin to prevent
backward movement, which should be sufficient since we don't guarantee
protection for rows deleted before slot creation.
---
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
src/backend/commands/subscriptioncmds.c | 70 +-----------
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 86 +++++++++------
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 101 +++++++-----------
src/include/replication/logicalworker.h | 3 +-
src/include/replication/worker_internal.h | 29 ++++-
src/test/subscription/t/035_conflicts.pl | 26 +----
9 files changed, 132 insertions(+), 194 deletions(-)
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index da9b559f18e..ac431034e0e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -262,9 +262,8 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
+ and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 80307319def..625e9e762b3 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -650,7 +650,7 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retainconflictinfo. */
CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
- !opts.enabled, WARNING);
+ !opts.enabled);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -1067,22 +1067,14 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover, two_phase, and retain_conflict_info
- * options.
+ * Common checks for altering failover and two_phase options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0 ||
- strcmp(option, "retain_conflict_info") == 0);
-
- /*
- * Altering the retain_conflict_info option does not update the slot on
- * the publisher.
- */
- Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
+ strcmp(option, "two_phase") == 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1094,41 +1086,6 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
- *
- * Additionally, do not allow changing the retain_conflict_info option
- * when the subscription is enabled to prevent race conditions arising
- * from the new option value being acknowledged asynchronously by the
- * launcher and apply workers.
- *
- * Without the restriction, a race condition may arise when a user
- * disables and immediately re-enables the retain_conflict_info option. In
- * this case, the launcher might drop the slot upon noticing the disabled
- * action, while the apply worker may keep maintaining
- * oldest_nonremovable_xid without noticing the option change. During this
- * period, a transaction ID wraparound could falsely make this ID appear
- * as if it originates from the future w.r.t the transaction ID stored in
- * the slot maintained by launcher.
- *
- * Similarly, if the user enables retain_conflict_info concurrently with
- * the launcher starting the worker, the apply worker may start
- * calculating oldest_nonremovable_xid before the launcher notices the
- * enable action. Consequently, the launcher may update slot.xmin to a
- * newer value than that maintained by the worker. In subsequent cycles,
- * upon integrating the worker's oldest_nonremovable_xid, the launcher
- * might detect a retreat in the calculated xmin, necessitating additional
- * handling.
- *
- * XXX To address the above race conditions, we can define
- * oldest_nonremovable_xid as FullTransactionID and adds the check to
- * disallow retreating the conflict slot's xmin. For now, we kept the
- * implementation simple by disallowing change to the
- * retain_conflict_info, but in the future we can change this after some
- * more analysis.
- *
- * Note that we could restrict only the enabling of retain_conflict_info
- * to avoid the race conditions described above, but we maintain the
- * restriction for both enable and disable operations for the sake of
- * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1403,28 +1360,12 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retainconflictinfo);
replaces[Anum_pg_subscription_subretainconflictinfo - 1] = true;
- CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
-
- /*
- * Workers may continue running even after the
- * subscription has been disabled.
- *
- * To prevent race conditions (as described in
- * CheckAlterSubOption()), ensure that all worker
- * processes have already exited before proceeding.
- */
- if (logicalrep_workers_find(subid, true, true))
- ereport(ERROR,
- (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
- errhint("Try again after some time.")));
-
/*
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
CheckSubConflictInfoRetention(opts.retainconflictinfo,
- true, !sub->enabled, NOTICE);
+ true, !sub->enabled);
/*
* Notify the launcher to manage the replication slot for
@@ -1476,8 +1417,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* comments atop CheckSubConflictInfoRetention() for details.
*/
CheckSubConflictInfoRetention(sub->retainconflictinfo,
- opts.enabled, !opts.enabled,
- WARNING);
+ opts.enabled, !opts.enabled);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index 1fa931a7422..d25085d3515 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,8 +441,7 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg),
- false);
+ dsm_segment_handle(winfo->dsm_seg));
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ca96f219405..5133854c9bc 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -105,11 +105,11 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
+ FullTransactionId *xmin,
bool *can_advance_xmin,
bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void advance_conflict_slot_xmin(FullTransactionId new_xmin);
static void invalidate_conflict_slot(void);
@@ -321,8 +321,7 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm,
- bool retain_conflict_info)
+ Oid relid, dsm_handle subworker_dsm)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -341,13 +340,10 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
- * - The replication slot used in conflict detection is created when
- * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
- Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -470,10 +466,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
- worker->oldest_nonremovable_xid = retain_conflict_info
- ? MyReplicationSlot->data.xmin
- : InvalidTransactionId;
- worker->stop_conflict_info_retention = (retain_conflict_info &&
+ worker->oldest_nonremovable_xid = InvalidFullTransactionId;
+ worker->stop_conflict_info_retention = (MyReplicationSlot &&
MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
@@ -1192,7 +1186,7 @@ ApplyLauncherMain(Datum main_arg)
bool can_advance_xmin = true;
bool retain_conflict_info = false;
bool stop_retention = true;
- TransactionId xmin = InvalidTransactionId;
+ FullTransactionId xmin = InvalidFullTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1265,8 +1259,7 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID,
- sub->retainconflictinfo))
+ DSM_HANDLE_INVALID))
{
/*
* We get here either if we failed to launch a worker
@@ -1346,7 +1339,7 @@ ApplyLauncherMain(Datum main_arg)
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
+ bool retain_conflict_info, FullTransactionId *xmin,
bool *can_advance_xmin, bool *stop_retention)
{
if (!retain_conflict_info)
@@ -1355,13 +1348,7 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (worker)
{
bool stop_conflict_info_retention;
- TransactionId nonremovable_xid;
-
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ FullTransactionId nonremovable_xid;
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
@@ -1381,10 +1368,14 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
if (stop_conflict_info_retention || !*can_advance_xmin)
return;
- Assert(TransactionIdIsValid(nonremovable_xid));
-
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
+ /*
+ * 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;
}
else
@@ -1449,19 +1440,48 @@ acquire_conflict_slot_if_exists(void)
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+advance_conflict_slot_xmin(FullTransactionId new_xmin)
{
+ FullTransactionId full_xmin;
+ FullTransactionId next_full_xid;
+
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(FullTransactionIdIsValid(new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
- if (TransactionIdEquals(MyReplicationSlot->data.xmin, 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);
+
+ /*
+ * Do not allow the xmin to go backwards. The newly computed xmin might be
+ * older than the current xmin if the slot was created after the apply
+ * worker began maintaining oldest_nonremovable_xid. This can occur if a
+ * user disables and immediately re-enables the retain_conflict_info
+ * option. In this case, the launcher might drop the slot upon noticing the
+ * disable action, while the apply worker may keep running with an old
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, if the some transaction IDs are assigned, the newly created slot
+ * will have a newer xmin compared to the worker.
+ *
+ * Similarily, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * regression in the calculated xmin, necessitating additional handling.
+ */
+ if (FullTransactionIdPrecedesOrEquals(new_xmin, full_xmin))
return;
SpinLockAcquire(&MyReplicationSlot->mutex);
- MyReplicationSlot->effective_xmin = new_xmin;
- MyReplicationSlot->data.xmin = new_xmin;
+ MyReplicationSlot->effective_xmin = XidFromFullTransactionId(new_xmin);
+ MyReplicationSlot->data.xmin = XidFromFullTransactionId(new_xmin);
SpinLockRelease(&MyReplicationSlot->mutex);
elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index accfa94badd..c90f23ee5b0 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,8 +615,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID,
- false);
+ DSM_HANDLE_INVALID);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 900ff7448fe..d783a2bb3b0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -313,8 +313,8 @@ typedef struct RetainConflictInfoData
* (RCI_WAIT_FOR_LOCAL_FLUSH). Use
* FullTransactionId for the same
* reason as remote_nextxid */
- TransactionId candidate_xid; /* candidate for the non-removable
- * transaction ID */
+ FullTransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
TimestampTz flushpos_update_time; /* when the remote flush position was
* updated in final phase
* (RCI_WAIT_FOR_LOCAL_FLUSH) */
@@ -487,8 +487,6 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
-static void apply_worker_exit(void);
-
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4227,6 +4225,8 @@ static void
get_candidate_xid(RetainConflictInfoData *rci_data)
{
TransactionId oldest_running_xid;
+ FullTransactionId next_full_xid;
+ FullTransactionId full_oldest_xid;
TimestampTz now;
/*
@@ -4254,17 +4254,20 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
rci_data->candidate_xid_time = now;
oldest_running_xid = GetOldestActiveTransactionId(false);
+ next_full_xid = ReadNextFullTransactionId();
/*
- * Oldest active transaction ID (oldest_running_xid) can't be behind any
- * of its previously computed value.
+ * Compute FullTransactionId for the oldest running transaction ID. This
+ * handles the case where transaction ID wraparound has occurred.
*/
- Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid));
+ full_oldest_xid = FullTransactionIdFromAllowableAt(next_full_xid, oldest_running_xid);
+
+ Assert(FullTransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid));
/* Return if the oldest_nonremovable_xid cannot be advanced */
- if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
- oldest_running_xid))
+ if (FullTransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ full_oldest_xid))
{
adjust_xid_advance_interval(rci_data, false);
return;
@@ -4272,7 +4275,7 @@ get_candidate_xid(RetainConflictInfoData *rci_data)
adjust_xid_advance_interval(rci_data, true);
- rci_data->candidate_xid = oldest_running_xid;
+ rci_data->candidate_xid = full_oldest_xid;
rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
/* process the next phase */
@@ -4381,7 +4384,7 @@ static void
wait_for_local_flush(RetainConflictInfoData *rci_data)
{
Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
- TransactionIdIsValid(rci_data->candidate_xid));
+ FullTransactionIdIsValid(rci_data->candidate_xid));
/*
* We expect the publisher and subscriber clocks to be in sync using time
@@ -4473,22 +4476,16 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
/*
* If conflict info retention was previously stopped due to a timeout, and
* the time required to advance the non-removable transaction ID has now
- * decreased to within acceptable limits, log a message and exit. This
- * allows the launcher to recreate the replication slot prior to
- * restarting the worker.
+ * decreased to within acceptable limits, log a message.
*/
if (MyLogicalRepWorker->stop_conflict_info_retention)
- {
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining conflict information",
MySubscription->name),
errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
max_conflict_retention_duration));
- apply_worker_exit();
- }
-
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4501,7 +4498,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rci_data->remote_lsn),
- rci_data->candidate_xid);
+ XidFromFullTransactionId(rci_data->candidate_xid));
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
@@ -4527,7 +4524,7 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_data)
rci_data->remote_nextxid = InvalidFullTransactionId;
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
- rci_data->candidate_xid = InvalidTransactionId;
+ rci_data->candidate_xid = InvalidFullTransactionId;
}
/*
@@ -4546,7 +4543,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
{
TimestampTz now;
- Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(FullTransactionIdIsValid(rci_data->candidate_xid));
Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
@@ -4581,7 +4578,7 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_data)
max_conflict_retention_duration));
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidFullTransactionId;
MyLogicalRepWorker->stop_conflict_info_retention = true;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
@@ -4738,6 +4735,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 was changed. 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 ||
@@ -4747,7 +4753,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,
@@ -5427,30 +5434,6 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
- /*
- * Restart the worker if retain_conflict_info was enabled during startup.
- *
- * At this point, the replication slot used for conflict detection might
- * not exist yet, or could be dropped soon if the launcher perceives
- * retain_conflict_info as disabled. To avoid unnecessary tracking of
- * oldest_nonremovable_xid when the slot is absent or at risk of being
- * dropped, a restart is initiated.
- *
- * The oldest_nonremovable_xid should be initialized only when the
- * retain_conflict_info is enabled before launching the worker. See
- * logicalrep_worker_launch.
- */
- if (am_leader_apply_worker() &&
- MySubscription->retainconflictinfo &&
- !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- {
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
- MySubscription->name, "retain_conflict_info"));
-
- apply_worker_exit();
- }
-
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -5613,7 +5596,7 @@ DisableSubscriptionAndExit(void)
* context.
*/
CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
- true, WARNING);
+ true);
proc_exit(0);
}
@@ -5991,18 +5974,13 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_conflict_info for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING if the subscription is disabled. Do not raise an ERROR since
+ * as long as the subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
- bool sub_disabled, int elevel_for_sub_disabled)
+ bool sub_disabled)
{
- Assert(elevel_for_sub_disabled == NOTICE ||
- elevel_for_sub_disabled == WARNING);
-
if (!retain_conflict_info)
return;
@@ -6020,10 +5998,9 @@ CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
"track_commit_timestamp"));
if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_conflict_info") : 0);
+ errhint("Consider setting %s to false.",
+ "retain_conflict_info"));
}
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index ebbb3ef09c7..2d006e7888e 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -32,7 +32,6 @@ extern void AtEOXact_LogicalRepWorkers(bool isCommit);
extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
bool check_guc,
- bool sub_disabled,
- int elevel_for_sub_disabled);
+ bool sub_disabled);
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 243164aada8..7cfc10d4052 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,8 +94,32 @@ typedef struct LogicalRepWorker
* 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
+ * w.r.t the transaction ID stored in the slot maintained by launcher. See
+ * advance_conflict_slot_xmin.
+ *
+ * Closing this race condition is complex. A potential solution involves
+ * the apply worker waiting for slot creation before updating
+ * oldest_nonremovable_xid. However, ensuring a race-free mechanism is
+ * difficult, especially when users concurrently toggle
+ * retain_conflict_info. We must prevent the launcher from prematurely
+ * recreating the slot if a subscription re-enables retain_conflict_info
+ * after it's been disabled and the slot dropped, and before workers reset
+ * their oldest_nonremovable_xid.
+ *
+ * Another approach could be to manage slot creation and deletion within
+ * subscription DDLs using strong locks to prevent race conditions.
+ * However, this method isn't entirely reliable. Commands can be rolled
+ * back, and even if slot creation is deferred until after all database
+ * modifications, transaction failures can still occur during the commit
+ * phase.
*/
- TransactionId oldest_nonremovable_xid;
+ FullTransactionId oldest_nonremovable_xid;
/*
* Indicates whether the apply worker has stopped retaining conflict
@@ -262,8 +286,7 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm,
- bool retain_conflict_info);
+ dsm_handle subworker_dsm);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 6c9c90c04d8..ca7f2b323d3 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -219,31 +219,13 @@ my $result = $node_B->safe_psql('postgres',
is($result, qq(t), 'worker on node B retains conflict information');
##################################################
-# Check that the retain_conflict_info option can be enabled only for disabled
-# subscriptions. Validate the NOTICE message during the subscription DDL, and
-# ensure the conflict detection slot is created upon enabling the
+# Check that the conflict detection slot is created after enabling
# retain_conflict_info option.
##################################################
-# Alter retain_conflict_info for enabled subscription
-my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
- "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
-ok( $stderr =~
- /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
- "altering retain_conflict_info is not allowed for enabled subscription");
-
-# Disable the subscription
-$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
-
-# Enable retain_conflict_info for disabled subscription
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+# Enable retain_conflict_info for the subscription
+$node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
-ok( $stderr =~
- /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
- "altering retain_conflict_info is allowed for disabled subscription");
-
-# Re-enable the subscription
-$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
# Confirm that the conflict detection slot is created on Node A and the xmin
# value is valid.
@@ -263,7 +245,7 @@ is($result, qq(t), 'worker on node A retains conflict information');
# other than the publisher.
##################################################
-($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (origin = any);");
ok( $stderr =~
/WARNING: subscription "tap_sub_a_b" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins/,
--
2.30.0.windows.2
v47-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v47-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 5a0cbde1fd19df18fca0b1344b09ca42351bbff0 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Fri, 4 Jul 2025 18:46:13 +0800
Subject: [PATCH v47 2/5] 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 | 43 ++++++
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 | 117 +++++++++++++++--
src/backend/replication/logical/worker.c | 124 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 341 insertions(+), 29 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0908f2e32f8..4c870b5e806 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5397,6 +5397,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_conflict_info</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_conflict_info</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_conflict_info</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_conflict_info</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 4265a22d4de..84e5a48181c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_conflict_info</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 82825db03bb..54cc1a501e7 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -2935,6 +2935,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 4d3de5bd979..5221454ef67 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -988,7 +988,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 dfbe268f959..ededd8d65a1 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;
@@ -103,9 +106,11 @@ static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
TransactionId *xmin,
- bool *can_advance_xmin);
+ bool *can_advance_xmin,
+ bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -468,6 +473,8 @@ retry:
worker->oldest_nonremovable_xid = retain_conflict_info
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_conflict_info &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1184,6 +1191,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_conflict_info = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1216,7 +1224,8 @@ ApplyLauncherMain(Datum main_arg)
* has set the retain_conflict_info option.
*/
compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
+ &xmin, &can_advance_xmin,
+ &stop_retention);
continue;
}
@@ -1229,7 +1238,7 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection.
*/
compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ &can_advance_xmin, &stop_retention);
if (w != NULL)
continue; /* worker is running already */
@@ -1278,14 +1287,19 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain the conflict information. Otherwise, if
- * required, advance the slot's xmin to protect deleted tuples
- * required for the conflict detection.
+ * Manage the replication slot based on requirements: - Invalidate the
+ * slot only if all workers for subscriptions with
+ * retain_conflict_info enabled have requested it. - Drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain the conflict information. - if required,
+ * advance the slot's xmin to protect deleted tuples required for the
+ * conflict detection.
*/
if (MyReplicationSlot)
{
- if (!retain_conflict_info)
+ if (retain_conflict_info && stop_retention)
+ invalidate_conflict_slot();
+ else if (!retain_conflict_info)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
advance_conflict_slot_xmin(xmin);
@@ -1326,17 +1340,21 @@ ApplyLauncherMain(Datum main_arg)
* If the replication slot cannot be advanced during this cycle, due to either
* a disabled subscription or an inactive worker, set *can_advance_xmin to
* false.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+ bool *can_advance_xmin, bool *stop_retention)
{
- if (!retain_conflict_info || !*can_advance_xmin)
+ if (!retain_conflict_info)
return;
if (worker)
{
+ bool stop_conflict_info_retention;
TransactionId nonremovable_xid;
/*
@@ -1347,8 +1365,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have
+ * stopped conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !*can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1377,6 +1409,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
* running.
*/
*can_advance_xmin = false;
+
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_conflict_info enabled have requested it.
+ */
+ *stop_retention = false;
}
}
@@ -1433,6 +1471,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_conflict_info enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1513,7 +1582,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;
@@ -1590,6 +1659,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 f83193b7781..7cfc05c6ff7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -459,6 +459,8 @@ static void request_publisher_status(RetainConflictInfoData *rci_data);
static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
bool status_received);
static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void reset_conflict_info_fields(RetainConflictInfoData *rci_data);
+static bool should_stop_conflict_info_retention(RetainConflictInfoData *rci_data);
static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
bool new_xid_found);
@@ -3871,7 +3873,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rci_data.phase == RCI_GET_CANDIDATE_XID &&
rci_data.xid_advance_interval)
wait_time = Min(wait_time, rci_data.xid_advance_interval);
@@ -4164,6 +4167,10 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4304,6 +4311,13 @@ wait_for_publisher_status(RetainConflictInfoData *rci_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
+ return;
+
if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
rci_data->remote_wait_for = rci_data->remote_nextxid;
@@ -4385,6 +4399,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
* have a WAL position greater than the rci_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ /*
+ * Reset the timer to prevent stopping conflict info retention due to
+ * time consumed during table synchronization. Given that table sync
+ * is an infrequent operation, it merits a time adjustment.
+ */
+ if (max_conflict_retention_duration)
+ rci_data->candidate_xid_time = rci_data->last_recv_time
+ ? rci_data->last_recv_time
+ : GetCurrentTimestamp();
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rci_data))
return;
/*
@@ -4410,7 +4443,10 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->flushpos_update_time = rci_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 < rci_data->remote_lsn)
return;
@@ -4430,12 +4466,21 @@ wait_for_local_flush(RetainConflictInfoData *rci_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_conflict_info_fields(rci_data);
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Reset all data fields of RetainConflictInfoData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_conflict_info_fields(RetainConflictInfoData *rci_data)
+{
rci_data->phase = RCI_GET_CANDIDATE_XID;
rci_data->remote_lsn = InvalidXLogRecPtr;
rci_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4443,9 +4488,64 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
rci_data->reply_time = 0;
rci_data->remote_wait_for = InvalidFullTransactionId;
rci_data->candidate_xid = InvalidTransactionId;
+}
- /* process the next phase */
- process_rci_phase_transition(rci_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 *rci_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rci_data->candidate_xid));
+ Assert(rci_data->phase == RCI_WAIT_FOR_PUBLISHER_STATUS ||
+ rci_data->phase == RCI_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rci_data->last_recv_time ? rci_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rci_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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_conflict_info_fields(rci_data);
+
+ return true;
}
/*
@@ -4478,6 +4578,10 @@ adjust_xid_advance_interval(RetainConflictInfoData *rci_data, bool new_xid_found
*/
rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rci_data->xid_advance_interval = Min(rci_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 2017d399961..8f31e494914 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"},
};
/*
@@ -1668,6 +1669,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();
}
@@ -1786,6 +1792,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;
}
@@ -2056,6 +2068,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 511dc32d519..3a8ce6b74a8 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 341f88adc87..bb6e880f2ae 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 f0943fa9b0a..3ed0b1be9a4 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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 b29453e8e4f..6e3007db5f0 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 9c8c7b9840b..52b5d7aa815 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 4fb317b3f85..436fe445d64 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -96,6 +96,12 @@ typedef struct LogicalRepWorker
*/
TransactionId 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 6cf828ca8d0..79ed5233edb 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,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 9b675b95407..f06fb77ad8d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -211,6 +211,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $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 the retain_conflict_info option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -246,6 +250,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$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');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_conflict_info is
# enabled. This warns of the possibility of receiving changes from origins
@@ -271,7 +279,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.30.0.windows.2
v47-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v47-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From a1469100166bf5c754aa3a7fe17e1c37efaac210 Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 11 Jun 2025 15:48:09 +0800
Subject: [PATCH v47 3/5] 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 | 10 ++++
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
3 files changed, 58 insertions(+), 21 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 4c870b5e806..504f7a01ef0 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5423,7 +5423,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_conflict_info</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index ededd8d65a1..ca96f219405 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1389,6 +1389,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker,
}
else
{
+ /*
+ * Drop the invalidated slot and re-create it if the worker for a
+ * subscription with retain_conflict_info enabled has stopped,
+ * indicating it might restart to request retention of information
+ * necessary for conflict detection.
+ */
+ if (MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/*
* Create a replication slot to retain information necessary for
* conflict detection such as dead tuples, commit timestamps, and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 7cfc05c6ff7..8bf21938698 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -487,6 +487,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4167,10 +4169,6 @@ can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
if (!MySubscription->retainconflictinfo)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4450,6 +4448,25 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
if (last_flushpos < rci_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4457,6 +4474,7 @@ wait_for_local_flush(RetainConflictInfoData *rci_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rci_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4498,9 +4516,8 @@ reset_conflict_info_fields(RetainConflictInfoData *rci_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 *rci_data)
@@ -4529,19 +4546,26 @@ should_stop_conflict_info_retention(RetainConflictInfoData *rci_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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_conflict_info_fields(rci_data);
--
2.30.0.windows.2
On Fri, Jul 4, 2025 at 4:48 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical replication
setups, such as (A -> B -> C). When retain_conflict_info is enabled on Node C,
it may not retain information necessary for conflict detection when applying
changes originally replicated from Node A. This happens because Node C only
waits for locally originated changes on Node B to be applied before advancing
the non-removable transaction ID.For example, Consider a logical replication setup as mentioned above : A -> B -> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no concurrent
transactions from Node B. It is unaware of Node A’s concurrent update.@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may already have
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.Note that, as per decoding logic Node C sees the commit timestamp of the update
as 9:00 (origin commit_ts from Node A), not 9:04 (commit time on Node B). In
this case, since the UPDATE's timestamp is earlier than the DELETE, Node C
should ideally detect an update_deleted conflict. However, it cannot, because
it no longer retains the deleted tuple.Even if Node C attempts to retrieve the latest WAL position from Node A, Node C
doesn't maintain any LSN which we could use to compare with it.This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a physical
standby, as the required transaction information from the original primary is
unavailable. Moreover, this limitation is relevant only when the subscription
origin option is set to ANY, as only in that case changes from other origins
can be replicated. Since retain_conflict_info is primarily useful for conflict
detection in bidirectional clusters where the origin option is set to NONE,
this limitation appears acceptable.Given these findings, to help users avoid unintended configurations, we plan to
issue a warning in scenarios where replicated changes may include origins other
than the direct publisher, similar to the existing checks in the
check_publications_origin() function.Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.
In this setup if we have A->B->C->A then after we implement conflict
resolution is it possible that node A will just left with (2,2),
because (1,11) will be deleted while applying the changes from Node C
whereas node C has detected the indirect conflicting update from Node
A as update missing and has inserted the row and it will left with
(1,11) and (2,2). So can it cause divergence as I explained here, or
it will not? If not then can you explain how?
--
Regards,
Dilip Kumar
Google
On Sat, Jul 5, 2025 at 2:26 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Jul 4, 2025 at 4:48 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical replication
setups, such as (A -> B -> C). When retain_conflict_info is enabled on Node C,
it may not retain information necessary for conflict detection when applying
changes originally replicated from Node A. This happens because Node C only
waits for locally originated changes on Node B to be applied before advancing
the non-removable transaction ID.For example, Consider a logical replication setup as mentioned above : A -> B -> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no concurrent
transactions from Node B. It is unaware of Node A’s concurrent update.@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may already have
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.Note that, as per decoding logic Node C sees the commit timestamp of the update
as 9:00 (origin commit_ts from Node A), not 9:04 (commit time on Node B). In
this case, since the UPDATE's timestamp is earlier than the DELETE, Node C
should ideally detect an update_deleted conflict. However, it cannot, because
it no longer retains the deleted tuple.Even if Node C attempts to retrieve the latest WAL position from Node A, Node C
doesn't maintain any LSN which we could use to compare with it.This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a physical
standby, as the required transaction information from the original primary is
unavailable. Moreover, this limitation is relevant only when the subscription
origin option is set to ANY, as only in that case changes from other origins
can be replicated. Since retain_conflict_info is primarily useful for conflict
detection in bidirectional clusters where the origin option is set to NONE,
this limitation appears acceptable.Given these findings, to help users avoid unintended configurations, we plan to
issue a warning in scenarios where replicated changes may include origins other
than the direct publisher, similar to the existing checks in the
check_publications_origin() function.Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.In this setup if we have A->B->C->A then after we implement conflict
resolution is it possible that node A will just left with (2,2),
because (1,11) will be deleted while applying the changes from Node C
whereas node C has detected the indirect conflicting update from Node
A as update missing and has inserted the row and it will left with
(1,11) and (2,2). So can it cause divergence as I explained here, or
it will not? If not then can you explain how?
Thinking further, I believe this will lead to data divergence.
However, in this specific setup, that's not a concern. If a user needs
to guarantee consistency across all nodes, they'll have to configure a
one-to-one publication-subscription relationship between each pair of
nodes: A to B and B to A, B to C and C to B, and A to C and C to A. In
a cascading setup, however, we cannot expect all nodes to contain
identical data. So I think I am fine with giving a WARNING what you
have done in your patch.
--
Regards,
Dilip Kumar
Google
Dear hackers,
As a confirmation purpose, I did performance testing with four workloads
we did before.
Highlights
==========
The retests on the latest patch set v46 show results consistent with previous
observations:
- There is no performance impact on the publisher side
- There is no performance impact on the subscriber side, if the workload is
running only on subscriber.
- The performance is reduced on the subscriber side (TPS reduction (~50%) [Test-03])
when retain_conflict_info=on and pgbench is running on both side. Because dead
tuple retention for conflict detection. If high workloads on the publisher,
the apply workers must wait for the amount of transactions with earlier
timestamps to be applied and flushed before advancing the non-removable XID
to remove dead tuples.
- Subscriber-side TPS improves when the workload on the publisher is reduced.
- Performance on the subscriber can also be improved by tuning the
max_conflict_retention_duration GUC properly.
Used source
===========
pgHead commit fd7d7b7191 + v46 patchset
Machine details
===============
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
01. pgbench on publisher
========================
The workload is mostly same as [1]/messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com.
Workload:
- Ran pgbench with 40 clients for the publisher.
- The duration was 120s, and the measurement was repeated 10 times.
(pubtest.tar.gz can run the same workload)
Test Scenarios & Results:
- pgHead : Median TPS = 39809.84925
- pgHead + patch : Median TPS = 40102.88108
Observation:
- No performance regression observed with the patch applied.
- The results were consistent across runs.
Detailed Results Table:
- each cell shows the TPS in each case.
- patch(ON) means patched and retain_conflict_info=ON is set.
run# pgHEAD pgHead+patch(ON)
1 40106.88834 40356.60039
2 39854.17244 40087.18077
3 39516.26983 40063.34688
4 39746.45715 40389.40549
5 40014.83857 40537.24
6 39819.26374 40016.78705
7 39800.43476 38774.9827
8 39884.2691 40163.35257
9 39753.11246 39902.02755
10 39427.2353 40118.58138
median 39809.84925 40102.88108
02. pgbench on subscriber
========================
The workload is mostly same as [2]/messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com.
Workload:
- Ran pgbench with 40 clients for the *subscriber*.
- The duration was 120s, and the measurement was repeated 10 times.
(subtest.tar.gz can run the same workload)
Test Scenarios & Results:
- pgHead : Median TPS = 41564.64591
- pgHead + patch : Median TPS = 41083.09555
Observation:
- No performance regression observed with the patch applied.
- The results were consistent across runs.
Detailed Results Table:
run# pgHEAD pgHead+patch(ON)
1 41605.88999 41106.93126
2 41555.76448 40975.9575
3 41505.76161 41223.92841
4 41722.50373 41049.52787
5 41400.48427 41262.15085
6 41386.47969 41059.25985
7 41679.7485 40916.93053
8 41563.60036 41178.82461
9 41565.69145 41672.41773
10 41765.11049 40958.73512
median 41564.64591 41083.09555
03. pgbench on both sides
========================
The workload is mostly same as [3]/messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com.
Workload:
- Ran pgbench with 40 clients for the *both side*.
- The duration was 120s, and the measurement was repeated 10 times.
(bothtest.tar.gz can run the same workload)
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 16799.67659
- pgHead + patch : Median TPS = 17338.38423
Subscriber:
- pgHead : Median TPS = 16552.60515
- pgHead + patch : Median TPS = 8367.133693
Observation:
- No performance regression observed on the publisher with the patch applied.
- The performance is reduced on the subscriber side (TPS reduction (~50%)) due
to dead tuple retention for the conflict detection
Detailed Results Table:
On publisher:
run# pgHEAD pgHead+patch(ON)
1 16735.53391 17369.89325
2 16957.01458 17077.96864
3 16838.07008 17480.08206
4 16743.67772 17531.00493
5 16776.74723 17511.4314
6 16784.73354 17235.76573
7 16871.63841 17255.04538
8 16814.61964 17460.33946
9 16903.14424 17024.77703
10 16556.05636 17306.87522
median 16799.67659 17338.38423
On subscriber:
run# pgHEAD pgHead+patch(ON)
1 16505.27302 8381.200661
2 16765.38292 8353.310973
3 16899.41055 8396.901652
4 16305.05353 8413.058805
5 16722.90536 8320.833085
6 16587.64864 8327.217432
7 16508.45076 8369.205438
8 16357.05337 8394.34603
9 16724.90296 8351.718212
10 16517.56167 8365.061948
median 16552.60515 8367.133693
04. pgbench on both side, and max_conflict_retention_duration was tuned
========================================================================
The workload is mostly same as [4]/messages/by-id/OSCPR01MB14966F39BE1732B9E433023BFF5E72@OSCPR01MB14966.jpnprd01.prod.outlook.com.
Workload:
- Initially ran pgbench with 40 clients for the *both side*.
- Set max_conflict_retention_duration = {60, 120}
- When the slot is invalidated on the subscriber side, stop the benchmark and
wait until the subscriber would be caught up. Then the number of clients on
the publisher would be half.
In this test the conflict slot could be invalidated as expected when the workload
on the publisher was high, and it would not get invalidated anymore after
reducing the workload. This shows even if the slot has been invalidated once,
users can continue to detect the update_deleted conflict by reduce the
workload on the publisher.
- Total period of the test was 900s for each cases.
(max_conflixt.tar.gz can run the same workload)
Observation:
-
- Parallelism of the publisher side is reduced till 15->7->3 and finally the
conflict slot is not invalidated.
- TPS on the subscriber side is improved when the concurrency was reduced.
This is because the dead tuple accumulation is reduced on subscriber due to
the reduced workload on the publisher.
- when publisher has Nclients=3, no regression in subscriber's TPS
Detailed Results Table:
For max_conflict_retention_duration = 60s
On publisher:
Nclients duration [s] TPS
15 72 14079.1
7 82 9307
3 446 4133.2
On subscriber:
Nclients duration [s] TPS
15 72 6827
15 81 7200
15 446 19129.4
For max_conflict_retention_duration = 120s
On publisher:
Nclients duration [s] TPS
15 162 17835.3
7 152 9503.8
3 283 4243.9
On subscriber:
Nclients duration [s] TPS
15 162 4571.8
15 152 4707
15 283 19568.4
Thanks Nisha-san and Hou-san for helping the work.
[1]: /messages/by-id/CABdArM5SpMyGvQTsX0-d=b+JAh0VQjuoyf9jFqcrQ3JLws5eOw@mail.gmail.com
[2]: /messages/by-id/TYAPR01MB5692B0182356F041DC9DE3B5F53E2@TYAPR01MB5692.jpnprd01.prod.outlook.com
[3]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
[4]: /messages/by-id/OSCPR01MB14966F39BE1732B9E433023BFF5E72@OSCPR01MB14966.jpnprd01.prod.outlook.com
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Fri, Jul 4, 2025 at 8:18 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical replication
setups, such as (A -> B -> C). When retain_conflict_info is enabled on Node C,
it may not retain information necessary for conflict detection when applying
changes originally replicated from Node A. This happens because Node C only
waits for locally originated changes on Node B to be applied before advancing
the non-removable transaction ID.For example, Consider a logical replication setup as mentioned above : A -> B -> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no concurrent
transactions from Node B. It is unaware of Node A’s concurrent update.@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may already have
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.Note that, as per decoding logic Node C sees the commit timestamp of the update
as 9:00 (origin commit_ts from Node A), not 9:04 (commit time on Node B). In
this case, since the UPDATE's timestamp is earlier than the DELETE, Node C
should ideally detect an update_deleted conflict. However, it cannot, because
it no longer retains the deleted tuple.Even if Node C attempts to retrieve the latest WAL position from Node A, Node C
doesn't maintain any LSN which we could use to compare with it.This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a physical
standby, as the required transaction information from the original primary is
unavailable. Moreover, this limitation is relevant only when the subscription
origin option is set to ANY, as only in that case changes from other origins
can be replicated. Since retain_conflict_info is primarily useful for conflict
detection in bidirectional clusters where the origin option is set to NONE,
this limitation appears acceptable.Given these findings, to help users avoid unintended configurations, we plan to
issue a warning in scenarios where replicated changes may include origins other
than the direct publisher, similar to the existing checks in the
check_publications_origin() function.Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.
I'm still reviewing the 0001 patch but let me share some comments and
questions I have so far:
---
It seems there is no place where we describe the overall idea of
reliably detecting update_deleted conflicts. The comment atop
maybe_advance_nonremovable_xid() describes why the implemented
algorithm works for that purpose but doesn't how it is implemented,
for example the relationship with pg_conflict_detection slot. I'm not
sure the long comment atop maybe_advance_nonremovable_xid() is the
right place as it seems to be a description beyond explaining
maybe_advance_nonremovable_xid() function. Probably we can move that
comment and explain the overall idea somewhere for example atop
worker.c or in README.
---
The new parameter name "retain_conflict_info" sounds to me like we
keep the conflict information somewhere that has expired at some time
such as how many times insert_exists or update_origin_differs
happened. How about choosing a name that indicates retain dead tuples
more explicitly for example retain_dead_tuples?
---
You mentioned in the previous email:
Furthermore, we tested running pgbench on both publisher and subscriber[3].
Some regression was observed in TPS on the subscriber, because workload on the
publisher is pretty high and the apply workers must wait for the amount of
transactions with earlier timestamps to be applied and flushed before advancing
the non-removable XID to remove dead tuples. This is the expected behavior of
this approach since the patch's main goal is to retain dead tuples for reliable
conflict detection.
Have you conducted any performance testing of a scenario where a
publisher replicates a large number of databases (say 64) to a
subscriber? I'm particularly interested in a configuration where
retain_conflict_info is set to true, and there are 64 apply workers
running on the subscriber side. In such a setup, even when running
pgbench exclusively on the publisher's databases, I suspect the
replication lag would likely increase quickly, as all apply workers on
the subscriber would be impacted by the overhead of retaining dead
tuples.
---
@@ -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
Why do we need to change the existing options' value?
---
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
I guess the "even" in the first sentence is not necessary.
---
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
I think this function is quite confusing for several reasons. For
instance, it's doing more things than described in the comments such
as trying to create the CONFLICT_DETECTION_SLOT if no worker is
passed. Also, one of the caller describes:
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT even if one of the subscriptions
+ * is not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
but it's unclear to me from the function name that it tries to create
the replication slot. Furthermore, in this path it doesn't actually
compute xmin. I guess we can try to create CONFLICT_DETECTION_SLOT in
the loop of "foreach(lc, sublist)" and set false to can_advance_xmin
if either the subscription is disabled or the worker is not running.
---
+ FullTransactionId remote_oldestxid; /* oldest transaction ID that was in
+ * the commit phase on the publisher.
+ * Use FullTransactionId to prevent
+ * issues with transaction ID
+ * wraparound, where a new
+ * remote_oldestxid could falsely
+ * appear to originate from the past
+ * and block advancement */
+ FullTransactionId remote_nextxid; /* next transaction ID to be assigned
+ * on the publisher. Use
+ * FullTransactionId for consistency
+ * and to allow straightforward
+ * comparisons with remote_oldestxid. */
I think it would be readable if we could write them above each field.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Sun, Jul 6, 2025 at 8:03 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:
Dear hackers,
As a confirmation purpose, I did performance testing with four workloads
we did before.
Thank you for doing the performance tests!
03. pgbench on both sides
========================
The workload is mostly same as [3].Workload:
- Ran pgbench with 40 clients for the *both side*.
- The duration was 120s, and the measurement was repeated 10 times.(bothtest.tar.gz can run the same workload)
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 16799.67659
- pgHead + patch : Median TPS = 17338.38423
Subscriber:
- pgHead : Median TPS = 16552.60515
- pgHead + patch : Median TPS = 8367.133693
My first impression is that 40 clients is a small number at which a
50% performance degradation occurs in 120s. Did you test how many
clients are required to trigger the same level performance regression
with retain_conflict_info = off?
04. pgbench on both side, and max_conflict_retention_duration was tuned
========================================================================
The workload is mostly same as [4].Workload:
- Initially ran pgbench with 40 clients for the *both side*.
- Set max_conflict_retention_duration = {60, 120}
- When the slot is invalidated on the subscriber side, stop the benchmark and
wait until the subscriber would be caught up. Then the number of clients on
the publisher would be half.
In this test the conflict slot could be invalidated as expected when the workload
on the publisher was high, and it would not get invalidated anymore after
reducing the workload. This shows even if the slot has been invalidated once,
users can continue to detect the update_deleted conflict by reduce the
workload on the publisher.
- Total period of the test was 900s for each cases.(max_conflixt.tar.gz can run the same workload)
Observation:
-
- Parallelism of the publisher side is reduced till 15->7->3 and finally the
conflict slot is not invalidated.
- TPS on the subscriber side is improved when the concurrency was reduced.
This is because the dead tuple accumulation is reduced on subscriber due to
the reduced workload on the publisher.
- when publisher has Nclients=3, no regression in subscriber's TPS
I think that users typically cannot control the amount of workloads in
production, meaning that once the performance regression starts to
happen the subscriber could enter the loop where invalidating the
slot, recovreing the performance, creating the slot, and having the
performance problem.
Detailed Results Table:
For max_conflict_retention_duration = 60s
On publisher:
Nclients duration [s] TPS
15 72 14079.1
7 82 9307
3 446 4133.2On subscriber:
Nclients duration [s] TPS
15 72 6827
15 81 7200
15 446 19129.4For max_conflict_retention_duration = 120s
On publisher:
Nclients duration [s] TPS
15 162 17835.3
7 152 9503.8
3 283 4243.9On subscriber:
Nclients duration [s] TPS
15 162 4571.8
15 152 4707
15 283 19568.4
What does each duration mean in these results? Can we interpret the
test case of max_conflict_retention_duration=120s that when 7 clients
and 15 clients are working on the publisher and the subscriber
respectively, the TPS on the subscriber was about one fourth (17835.3
vs. 4707)?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Sun, Jul 6, 2025 at 10:51 PM Masahiko Sawada wrote:
On Fri, Jul 4, 2025 at 8:18 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical
replication setups, such as (A -> B -> C). When retain_conflict_info
is enabled on Node C, it may not retain information necessary for
conflict detection when applying changes originally replicated from
Node A. This happens because Node C only waits for locally originated
changes on Node B to be applied before advancing the non-removabletransaction ID.
For example, Consider a logical replication setup as mentioned above : A -> B
-> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no
concurrent transactions from Node B. It is unaware of Node A’s concurrentupdate.
@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may alreadyhave
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.Note that, as per decoding logic Node C sees the commit timestamp of
the update as 9:00 (origin commit_ts from Node A), not 9:04 (commit
time on Node B). In this case, since the UPDATE's timestamp is earlier
than the DELETE, Node C should ideally detect an update_deleted
conflict. However, it cannot, because it no longer retains the deleted tuple.Even if Node C attempts to retrieve the latest WAL position from Node
A, Node C doesn't maintain any LSN which we could use to compare with it.This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a
physical standby, as the required transaction information from the
original primary is unavailable. Moreover, this limitation is relevant
only when the subscription origin option is set to ANY, as only in
that case changes from other origins can be replicated. Since
retain_conflict_info is primarily useful for conflict detection in
bidirectional clusters where the origin option is set to NONE, this limitationappears acceptable.
Given these findings, to help users avoid unintended configurations,
we plan to issue a warning in scenarios where replicated changes may
include origins other than the direct publisher, similar to the
existing checks in the
check_publications_origin() function.Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.I'm still reviewing the 0001 patch but let me share some comments and
questions I have so far:
Thanks for the comments!
---
It seems there is no place where we describe the overall idea of reliably
detecting update_deleted conflicts. The comment atop
maybe_advance_nonremovable_xid() describes why the implemented
algorithm works for that purpose but doesn't how it is implemented, for
example the relationship with pg_conflict_detection slot. I'm not sure the long
comment atop maybe_advance_nonremovable_xid() is the right place as it
seems to be a description beyond explaining
maybe_advance_nonremovable_xid() function. Probably we can move that
comment and explain the overall idea somewhere for example atop worker.c or
in README.
I think it makes sense to explain it atop of worker.c, will do.
---
The new parameter name "retain_conflict_info" sounds to me like we keep the
conflict information somewhere that has expired at some time such as how
many times insert_exists or update_origin_differs happened. How about
choosing a name that indicates retain dead tuples more explicitly for example
retain_dead_tuples?
We considered the name you suggested, but we wanted to convey that this option
not only retains dead tuples but also preserves commit timestamps and origin
data for conflict detection, hence we opted for a more general name. Do you
have better suggestions?
---
You mentioned in the previous email:Furthermore, we tested running pgbench on both publisher and
subscriber[3].
Some regression was observed in TPS on the subscriber, because
workload on the publisher is pretty high and the apply workers must
wait for the amount of transactions with earlier timestamps to be
applied and flushed before advancing the non-removable XID to remove
dead tuples. This is the expected behavior of this approach since the
patch's main goal is to retain dead tuples for reliable conflict detection.Have you conducted any performance testing of a scenario where a publisher
replicates a large number of databases (say 64) to a subscriber? I'm particularly
interested in a configuration where retain_conflict_info is set to true, and there
are 64 apply workers running on the subscriber side. In such a setup, even
when running pgbench exclusively on the publisher's databases, I suspect the
replication lag would likely increase quickly, as all apply workers on the
subscriber would be impacted by the overhead of retaining dead tuples.
We will try this workload and share the feedback.
--- @@ -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 0x00010000Why do we need to change the existing options' value?
The intention is to position the new option after the failover option, ensuring
consistency with the order in the pg_subscription catalog. I think modifying existing
options in a major version is acceptable, as we have done similarly in commits
4826759 and 776621a.
--- + * This is required to ensure that we don't advance the xmin + * of CONFLICT_DETECTION_SLOT even if one of the subscriptions + * is not enabled. Otherwise, we won't be able to detectI guess the "even" in the first sentence is not necessary.
Agreed.
--- +/* + * Determine the minimum non-removable transaction ID across all apply +workers + * for subscriptions that have retain_conflict_info enabled. Store the +result + * in *xmin. + * + * If the replication slot cannot be advanced during this cycle, due to +either + * a disabled subscription or an inactive worker, set *can_advance_xmin +to + * false. + */ +static void +compute_min_nonremovable_xid(LogicalRepWorker *worker, + bool retain_conflict_info, TransactionId *xmin, + bool *can_advance_xmin)I think this function is quite confusing for several reasons. For instance, it's
doing more things than described in the comments such as trying to create the
CONFLICT_DETECTION_SLOT if no worker is passed. Also, one of the caller
describes:+ /* + * This is required to ensure that we don't advance the xmin + * of CONFLICT_DETECTION_SLOT even if one of the subscriptions + * is not enabled. Otherwise, we won't be able to detect + * conflicts reliably for such a subscription even though it + * has set the retain_conflict_info option. + */ + compute_min_nonremovable_xid(NULL, sub->retainconflictinfo, + &xmin, &can_advance_xmin);but it's unclear to me from the function name that it tries to create the
replication slot. Furthermore, in this path it doesn't actually compute xmin. I
guess we can try to create CONFLICT_DETECTION_SLOT in the loop of
"foreach(lc, sublist)" and set false to can_advance_xmin if either the
subscription is disabled or the worker is not running.
I understand. The original code was similar to your suggestion, but we decided
to encapsulate it within a separate function to maintain a clean and concise
main loop. However, your suggestion also makes sense, so I will proceed with
the change.
--- + FullTransactionId remote_oldestxid; /* oldest transaction ID that was in + * the commit phase on the publisher. + * Use FullTransactionId to prevent + * issues with transaction ID + * wraparound, where a new + * remote_oldestxid could falsely + * appear to originate from the past + * and block advancement */ + FullTransactionId remote_nextxid; /* next transaction ID to be assigned + * on the publisher. Use + * FullTransactionId for consistency + * and to allow straightforward + * comparisons with + remote_oldestxid. */I think it would be readable if we could write them above each field.
Will adjust.
Best Regards,
Hou zj
On Sun, Jul 6, 2025 at 10:51 PM Masahiko Sawada wrote:
On Sun, Jul 6, 2025 at 8:03 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:Dear hackers,
As a confirmation purpose, I did performance testing with four
workloads we did before.Thank you for doing the performance tests!
03. pgbench on both sides
========================
The workload is mostly same as [3].Workload:
- Ran pgbench with 40 clients for the *both side*.
- The duration was 120s, and the measurement was repeated 10 times.(bothtest.tar.gz can run the same workload)
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 16799.67659
- pgHead + patch : Median TPS = 17338.38423
Subscriber:
- pgHead : Median TPS = 16552.60515
- pgHead + patch : Median TPS = 8367.133693My first impression is that 40 clients is a small number at which a 50%
performance degradation occurs in 120s. Did you test how many clients are
required to trigger the same level performance regression with
retain_conflict_info = off?
Could you please elaborate further on the intention behind the suggested tests
and what outcomes are expected? I ask because we anticipate that disabling
retain_conflict_info should not cause regression, given that dead tuples will
not be retained.
04. pgbench on both side, and max_conflict_retention_duration was
tuned================================================
========================
The workload is mostly same as [4].Workload:
- Initially ran pgbench with 40 clients for the *both side*.
- Set max_conflict_retention_duration = {60, 120}
- When the slot is invalidated on the subscriber side, stop the benchmarkand
wait until the subscriber would be caught up. Then the number of clients
on
the publisher would be half.
In this test the conflict slot could be invalidated as expected when theworkload
on the publisher was high, and it would not get invalidated anymore after
reducing the workload. This shows even if the slot has been invalidatedonce,
users can continue to detect the update_deleted conflict by reduce the
workload on the publisher.
- Total period of the test was 900s for each cases.(max_conflixt.tar.gz can run the same workload)
Observation:
-
- Parallelism of the publisher side is reduced till 15->7->3 and finally the
conflict slot is not invalidated.
- TPS on the subscriber side is improved when the concurrency wasreduced.
This is because the dead tuple accumulation is reduced on subscriber
due to
the reduced workload on the publisher.
- when publisher has Nclients=3, no regression in subscriber's TPSI think that users typically cannot control the amount of workloads in
production, meaning that once the performance regression starts to happen
the subscriber could enter the loop where invalidating the slot, recovreing the
performance, creating the slot, and having the performance problem.
Yes, you are right. The test is designed to demonstrate that the slot can
be invalidated under high workload conditions as expected, while
remaining valid if the workload is reduced. In production systems where
workload reduction may not be possible, it’s recommended to disable
`retain_conflict_info` to enhance performance. This decision involves balancing
the need for reliable conflict detection with optimal system performance.
I think the hot standby feedback also has a similar impact on the performance
of the primary, which is done to prevent the early removal of data necessary
for the standby, ensuring that it remains accessible when needed.
Best Regards,
Hou zj
Dear Sawada-san,
What does each duration mean in these results? Can we interpret the
test case of max_conflict_retention_duration=120s that when 7 clients
and 15 clients are working on the publisher and the subscriber
respectively, the TPS on the subscriber was about one fourth (17835.3
vs. 4707)?
Firstly, this workload is done to prove that users can tune their workload to keep
enabling the update_deleted detections. Let me describe what happened there with
the timetable since the test starts.
0-162s:
Number of clients on both publisher/subscriber was 15. TPS was 17835.3 on the
publisher and 4571.8 on the subscriber. This means that retained dead tuples on
the subscriber may reduce the performance to around 1/4 compared with publisher,
and the workload on the publisher is too heavy to keep working the update_deleted
detection.
163-314s:
Number of clients was 7 on publisher, and 15 on subscriber. TPS was 9503.8 on
the publisher and 4707 on the subscriber. This means that N=7 on the publisher
was still too many thus conflict slot must be invalidated.
315-597s:
Number of clients was 3 on publisher, and 15 on subscriber. TPS was 4243.9 on
the publisher and 19568.4 on the subscriber. Here the conflict slot could survive
during the benchmark because concurrency on the publisher was reduced.
Performance could be improved on the subscriber side because dead tuples can be
reduced here.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Mon, Jul 7, 2025 at 11:03 AM Zhijie Hou (Fujitsu) wrote:
On Sun, Jul 6, 2025 at 10:51 PM Masahiko Sawada wrote:
======================================================================
==
The workload is mostly same as [4].Workload:
- Initially ran pgbench with 40 clients for the *both side*.
- Set max_conflict_retention_duration = {60, 120}
- When the slot is invalidated on the subscriber side, stop the
benchmarkand
wait until the subscriber would be caught up. Then the number of
clientson
the publisher would be half.
In this test the conflict slot could be invalidated as expected
when theworkload
on the publisher was high, and it would not get invalidated anymore
after
reducing the workload. This shows even if the slot has been
invalidatedonce,
users can continue to detect the update_deleted conflict by reduce the
workload on the publisher.
- Total period of the test was 900s for each cases.(max_conflixt.tar.gz can run the same workload)
Observation:
-
- Parallelism of the publisher side is reduced till 15->7->3 and finally the
conflict slot is not invalidated.
- TPS on the subscriber side is improved when the concurrency wasreduced.
This is because the dead tuple accumulation is reduced on
subscriberdue to
the reduced workload on the publisher.
- when publisher has Nclients=3, no regression in subscriber's TPSI think that users typically cannot control the amount of workloads in
production, meaning that once the performance regression starts to
happen the subscriber could enter the loop where invalidating the
slot, recovreing the performance, creating the slot, and having theperformance problem.
Yes, you are right. The test is designed to demonstrate that the slot can be
invalidated under high workload conditions as expected, while remaining valid
if the workload is reduced. In production systems where workload reduction
may not be possible, it’s recommended to disable `retain_conflict_info` to
enhance performance. This decision involves balancing the need for reliable
conflict detection with optimal system performance.I think the hot standby feedback also has a similar impact on the performance
of the primary, which is done to prevent the early removal of data necessary for
the standby, ensuring that it remains accessible when needed.
For reference, we conducted test[1]/messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com to evaluate the impact of enabling hot
standby feedback in a physical replication setup, observing approximately
a 50% regression in TPS on the primary as well.
[1]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
Best Regards,
Hou zj
On Mon, Jul 7, 2025 at 10:13 AM Zhijie Hou (Fujitsu) wrote:
On Sun, Jul 6, 2025 at 10:51 PM Masahiko Sawada wrote:
On Fri, Jul 4, 2025 at 8:18 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical
replication setups, such as (A -> B -> C). When retain_conflict_info
is enabled on Node C, it may not retain information necessary for
conflict detection when applying changes originally replicated from
Node A. This happens because Node C only waits for locally
originated changes on Node B to be applied before advancing the
non-removabletransaction ID.
For example, Consider a logical replication setup as mentioned above
: A -> B-> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no
concurrent transactions from Node B. It is unaware of Node A’s
concurrentupdate.
@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may
alreadyhave
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.Note that, as per decoding logic Node C sees the commit timestamp of
the update as 9:00 (origin commit_ts from Node A), not 9:04 (commit
time on Node B). In this case, since the UPDATE's timestamp is
earlier than the DELETE, Node C should ideally detect an
update_deleted conflict. However, it cannot, because it no longer retainsthe deleted tuple.
Even if Node C attempts to retrieve the latest WAL position from
Node A, Node C doesn't maintain any LSN which we could use to comparewith it.
This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a
physical standby, as the required transaction information from the
original primary is unavailable. Moreover, this limitation is
relevant only when the subscription origin option is set to ANY, as
only in that case changes from other origins can be replicated.
Since retain_conflict_info is primarily useful for conflict
detection in bidirectional clusters where the origin option is set
to NONE, this limitationappears acceptable.
Given these findings, to help users avoid unintended configurations,
we plan to issue a warning in scenarios where replicated changes may
include origins other than the direct publisher, similar to the
existing checks in the
check_publications_origin() function.Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.I'm still reviewing the 0001 patch but let me share some comments and
questions I have so far:Thanks for the comments!
--- +/* + * Determine the minimum non-removable transaction ID across all +apply workers + * for subscriptions that have retain_conflict_info enabled. Store +the result + * in *xmin. + * + * If the replication slot cannot be advanced during this cycle, due +to either + * a disabled subscription or an inactive worker, set +*can_advance_xmin to + * false. + */ +static void +compute_min_nonremovable_xid(LogicalRepWorker *worker, + bool retain_conflict_info, TransactionId*xmin,
+ bool *can_advance_xmin)
I think this function is quite confusing for several reasons. For
instance, it's doing more things than described in the comments such
as trying to create the CONFLICT_DETECTION_SLOT if no worker is
passed. Also, one of the caller
describes:+ /* + * This is required to ensure that we don't advance the xmin + * of CONFLICT_DETECTION_SLOT even if one of the subscriptions + * is not enabled. Otherwise, we won't be able to detect + * conflicts reliably for such a subscription even though it + * has set the retain_conflict_info option. + */ + compute_min_nonremovable_xid(NULL, sub->retainconflictinfo, + &xmin, + &can_advance_xmin);but it's unclear to me from the function name that it tries to create
the replication slot. Furthermore, in this path it doesn't actually
compute xmin. I guess we can try to create CONFLICT_DETECTION_SLOT in
the loop of "foreach(lc, sublist)" and set false to can_advance_xmin
if either the subscription is disabled or the worker is not running.I understand. The original code was similar to your suggestion, but we decided
to encapsulate it within a separate function to maintain a clean and concise
main loop. However, your suggestion also makes sense, so I will proceed with
the change.
I have made this change in the 0002 patch for reference. What do you think ? If
there are no objections, I plan to merge it in the next version.
Other comment adjustments that do not alter the logic have already been
merged into the 0001 patch.
Best Regards,
Hou zj
Attachments:
v48-0002-refactor-launcher-slot-creation.patchapplication/octet-stream; name=v48-0002-refactor-launcher-slot-creation.patchDownload
From 39e353c229c8a29fd6efce0f262475482b45190c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 7 Jul 2025 14:28:45 +0800
Subject: [PATCH v48 2/4] refactor launcher slot creation
---
src/backend/replication/logical/launcher.c | 121 ++++++++++-----------
1 file changed, 58 insertions(+), 63 deletions(-)
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f18fee0333e..e0f101784c7 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,8 +102,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
- bool *can_advance_xmin);
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
@@ -1204,22 +1203,41 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- retain_conflict_info |= sub->retainconflictinfo;
-
- if (!sub->enabled)
+ if (sub->retainconflictinfo)
{
+ retain_conflict_info = true;
+
/*
- * This is required to ensure that we don't advance the xmin
- * of CONFLICT_DETECTION_SLOT if one of the subscriptions is
- * not enabled. Otherwise, we won't be able to detect
- * conflicts reliably for such a subscription even though it
- * has set the retain_conflict_info option.
+ * Can't advance xmin of the slot unless all the subscriptions
+ * with retain_conflict_info are enabled. This is required to
+ * ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_conflict_info option.
*/
- compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
- continue;
+ can_advance_xmin &= sub->enabled;
+
+ /*
+ * Create a replication slot to retain information necessary
+ * for conflict detection such as dead tuples, commit
+ * timestamps, and origins.
+ *
+ * The slot is created before starting the apply worker to
+ * prevent it from unnecessarily maintaining its
+ * oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to
+ * ensure that conflict-related information is available when
+ * applying remote changes that occurred before the
+ * subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1228,12 +1246,20 @@ ApplyLauncherMain(Datum main_arg)
* Compute the minimum xmin required to protect deleted tuples
* required for conflict detection.
*/
- compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ if (can_advance_xmin)
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin);
if (w != NULL)
continue; /* worker is running already */
+ /*
+ * Can't advance xmin of the slot unless all the workers
+ * corresponding to subscriptions with retain_conflict_info are
+ * running.
+ */
+ if (sub->retainconflictinfo)
+ can_advance_xmin = false;
+
/*
* If the worker is eligible to start now, launch it. Otherwise,
* adjust wait_time so that we'll wake up as soon as it can be
@@ -1322,62 +1348,31 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_conflict_info enabled. Store the result
* in *xmin.
- *
- * If the replication slot cannot be advanced during this cycle, due to either
- * a disabled subscription or an inactive worker, set *can_advance_xmin to
- * false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool retain_conflict_info,
+ TransactionId *xmin)
{
- if (!retain_conflict_info || !*can_advance_xmin)
- return;
+ TransactionId nonremovable_xid;
- if (worker)
- {
- TransactionId nonremovable_xid;
+ if (!retain_conflict_info || !worker)
+ return;
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ /*
+ * The replication slot for conflict detection must be created before the
+ * worker starts.
+ */
+ Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
- nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ Assert(TransactionIdIsValid(nonremovable_xid));
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
- *xmin = nonremovable_xid;
- }
- else
- {
- /*
- * Create a replication slot to retain information necessary for
- * conflict detection such as dead tuples, commit timestamps, and
- * origins.
- *
- * The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid.
- *
- * The slot is created even for a disabled subscription to ensure that
- * conflict-related information is available when applying remote
- * changes that occurred before the subscription was enabled.
- */
- CreateConflictDetectionSlot();
-
- /*
- * Can't advance xmin of the slot unless all the subscriptions with
- * retain_conflict_info are enabled and the corresponding workers are
- * running.
- */
- *can_advance_xmin = false;
- }
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
}
/*
--
2.30.0.windows.2
v48-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v48-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From d20ac6168d7374368ed13d60d959ae2340bcca8c Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Wed, 25 Jun 2025 10:28:04 +0800
Subject: [PATCH v48 1/4] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 50 +-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 357 ++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 221 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 666 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 +++--
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 195 ++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2180 insertions(+), 215 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 4f9192316e0..de9617e2893 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 59a0874528a..0908f2e32f8 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4963,6 +4963,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index 810b2b50f0d..1d593079fd9 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29592,7 +29592,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29636,7 +29638,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29666,6 +29670,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29688,8 +29694,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index c32e6bc000d..2394f282253 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2364,6 +2364,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2500,6 +2506,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2531,6 +2553,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index 82fe3f93761..29ffa343d5f 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2225,6 +2225,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2643,6 +2645,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2787,6 +2848,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..7782246727e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,52 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+
+ <para>
+ Enabling this option ensures retention of information useful for
+ conflict detection solely for changes occurring locally on the
+ publisher. For the changes originating from different origins,
+ reliable conflict detection cannot be guaranteed.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 8975dc6d3c8..bfdeec70606 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1183,7 +1183,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 in logical replication. See
+ * GetOldestActiveTransactionId(true) and its use.
*
* We save the PREPARE record's location in the gxact for later use by
* CheckPointTwoPhase.
@@ -2298,7 +2302,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2318,7 +2322,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2331,8 +2335,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 comments atop worker.c.
+ */
+ committs = GetCurrentTimestamp();
/*
* Emit the XLOG commit record. Note that we mark 2PC commits as
@@ -2381,7 +2401,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 41601fcb280..b46e7e9c2a6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 47ffc0a2307..e116745fb1a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7142,7 +7142,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 93d38914854..2308b98ef73 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 e5dbbe61b81..4d3de5bd979 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1378,7 +1378,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 4ff246cd943..4a9e65e49a6 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;
@@ -105,8 +108,11 @@ typedef struct SubOpts
static List *fetch_table_list(WalReceiverConn *wrconn, List *publications);
static void check_publications_origin(WalReceiverConn *wrconn,
List *publications, bool copydata,
- char *origin, Oid *subrel_local_oids,
- int subrel_count, char *subname);
+ bool retain_conflict_info, char *origin,
+ Oid *subrel_local_oids, int subrel_count,
+ char *subname);
+static void check_pub_conflict_info_retention(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 +168,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);
@@ -210,7 +218,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)
@@ -307,6 +315,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 +580,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);
/*
@@ -630,6 +648,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +692,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)
@@ -722,7 +746,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
{
check_publications(wrconn, publications);
check_publications_origin(wrconn, publications, opts.copy_data,
- opts.origin, NULL, 0, stmt->subname);
+ opts.retainconflictinfo, opts.origin,
+ NULL, 0, stmt->subname);
+
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
/*
* Set sync state based on if we were asked to do data copy or
@@ -881,8 +908,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
sizeof(Oid), oid_cmp);
check_publications_origin(wrconn, sub->publications, copy_data,
- sub->origin, subrel_local_oids,
- subrel_count, sub->name);
+ sub->retainconflictinfo, sub->origin,
+ subrel_local_oids, subrel_count, sub->name);
/*
* Rels that we want to remove from subscription and drop any slots
@@ -1040,18 +1067,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1094,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1176,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_tuple = false;
bool update_failover = false;
bool update_two_phase = false;
+ bool check_pub_rci = false;
+ bool retain_conflict_info;
+ char *origin;
Subscription *sub;
Form_pg_subscription form;
bits32 supported_opts;
@@ -1137,6 +1206,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
sub = GetSubscription(subid, false);
+ retain_conflict_info = sub->retainconflictinfo;
+ origin = sub->origin;
+
/*
* Don't allow non-superuser modification of a subscription with
* password_required=false.
@@ -1165,7 +1237,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,11 +1397,62 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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();
+
+ check_pub_rci = opts.retainconflictinfo;
+ retain_conflict_info = opts.retainconflictinfo;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
CStringGetTextDatum(opts.origin);
replaces[Anum_pg_subscription_suborigin - 1] = true;
+
+ /*
+ * Check if changes from different origins may be received
+ * from the publisher when the origin is changed to ANY
+ * and retain_conflict_info is enabled.
+ */
+ check_pub_rci = retain_conflict_info &&
+ pg_strcasecmp(opts.origin, LOGICALREP_ORIGIN_ANY) == 0;
+
+ origin = opts.origin;
}
update_tuple = true;
@@ -1347,6 +1470,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1487,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.
+ */
+ check_pub_rci = sub->retainconflictinfo && opts.enabled;
break;
}
@@ -1369,6 +1509,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.
+ */
+ check_pub_rci = sub->retainconflictinfo;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1715,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 || check_pub_rci)
{
bool must_use_password;
char *err;
@@ -1584,10 +1732,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 +1748,16 @@ 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_pub_conflict_info_retention(wrconn, retain_conflict_info);
+
+ check_publications_origin(wrconn, sub->publications, false,
+ retain_conflict_info, origin, NULL, 0,
+ sub->name);
+
+ 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();
{
@@ -2086,20 +2245,29 @@ AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId)
* Check and log a warning if the publisher has subscribed to the same table,
* its partition ancestors (if it's a partition), or its partition children (if
* it's a partitioned table), from some other publishers. This check is
- * required only if "copy_data = true" and "origin = none" for CREATE
- * SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements to notify the
- * user that data having origin might have been copied.
+ * required in the following scenarios:
*
- * This check need not be performed on the tables that are already added
- * because incremental sync for those tables will happen through WAL and the
- * origin of the data can be identified from the WAL records.
+ * 1) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "copy_data = true" and "origin = none":
+ * - Warn the user that data with an origin might have been copied.
+ * - This check is skipped for tables already added, as incremental sync via
+ * WAL allows origin tracking. The list of such tables is in
+ * subrel_local_oids.
*
- * subrel_local_oids contains the list of relation oids that are already
- * present on the subscriber.
+ * 2) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "retain_conflict_info = true" and "origin = any", and for ALTER
+ * SUBSCRIPTION statements that modify retain_conflict_info or origin, or
+ * when the publisher's status changes (e.g., due to a connection string
+ * update):
+ * - Warn the user that only conflict detection info for local changes on
+ * the publisher is retained. Data from other origins may lack sufficient
+ * details for reliable conflict detection.
+ * - See comments atop worker.c for more details.
*/
static void
check_publications_origin(WalReceiverConn *wrconn, List *publications,
- bool copydata, char *origin, Oid *subrel_local_oids,
+ bool copydata, bool retain_conflict_info,
+ char *origin, Oid *subrel_local_oids,
int subrel_count, char *subname)
{
WalRcvExecResult *res;
@@ -2108,9 +2276,29 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
Oid tableRow[1] = {TEXTOID};
List *publist = NIL;
int i;
+ bool check_rci;
+ bool check_table_sync;
+ bool origin_none = origin &&
+ pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) == 0;
+
+ /*
+ * Enable retain_conflict_info checks only when origin is set to 'any',
+ * since with origin='none' only local changes are replicated to the
+ * subscriber.
+ */
+ check_rci = retain_conflict_info && !origin_none;
- if (!copydata || !origin ||
- (pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) != 0))
+ /*
+ * Enable table synchronization checks only when origin is 'none', to
+ * ensure that data from other origins is not inadvertently copied.
+ */
+ check_table_sync = copydata && origin_none;
+
+ /* retain_conflict_info and table sync checks occur separately */
+ Assert(!(check_rci && check_table_sync));
+
+ /* Return if no checks are required */
+ if (!check_rci && !check_table_sync)
return;
initStringInfo(&cmd);
@@ -2129,16 +2317,23 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
/*
* In case of ALTER SUBSCRIPTION ... REFRESH, subrel_local_oids contains
* the list of relation oids that are already present on the subscriber.
- * This check should be skipped for these tables.
+ * This check should be skipped for these tables if checking for table
+ * sync scenario. However, when handling the retain_conflict_info
+ * scenario, ensure all tables are checked, as some existing tables may
+ * now include changes from other origins due to newly created
+ * subscriptions on the publisher.
*/
- for (i = 0; i < subrel_count; i++)
+ if (check_table_sync)
{
- Oid relid = subrel_local_oids[i];
- char *schemaname = get_namespace_name(get_rel_namespace(relid));
- char *tablename = get_rel_name(relid);
+ for (i = 0; i < subrel_count; i++)
+ {
+ Oid relid = subrel_local_oids[i];
+ char *schemaname = get_namespace_name(get_rel_namespace(relid));
+ char *tablename = get_rel_name(relid);
- appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
- schemaname, tablename);
+ appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
+ schemaname, tablename);
+ }
}
res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
@@ -2173,22 +2368,37 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
* XXX: For simplicity, we don't check whether the table has any data or
* not. If the table doesn't have any data then we don't need to
* distinguish between data having origin and data not having origin so we
- * can avoid logging a warning in that case.
+ * can avoid logging a warning for table sync scenario.
*/
if (publist)
{
StringInfo pubnames = makeStringInfo();
+ StringInfo err_msg = makeStringInfo();
+ StringInfo err_hint = makeStringInfo();
/* Prepare the list of publication(s) for warning message. */
GetPublicationsStr(publist, pubnames, false);
+
+ if (check_table_sync)
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin"),
+ subname);
+ appendStringInfoString(err_hint, _("Verify that initial data copied from the publisher tables did not come from other origins."));
+ }
+ else
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins"),
+ subname);
+ appendStringInfoString(err_hint, _("Consider using origin = NONE or disabling retain_conflict_info."));
+ }
+
ereport(WARNING,
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin",
- subname),
- errdetail_plural("The subscription being created subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
- "The subscription being created subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
+ errmsg_internal("%s", err_msg->data),
+ errdetail_plural("The subscription subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
+ "The subscription subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
list_length(publist), pubnames->data),
- errhint("Verify that initial data copied from the publisher tables did not come from other origins."));
+ errhint_internal("%s", err_hint->data));
}
ExecDropSingleTupleTableSlot(slot);
@@ -2196,6 +2406,57 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop worker.c for a detailed explanation.
+ */
+static void
+check_pub_conflict_info_retention(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) < 19000)
+ 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 19"));
+
+ 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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..f18fee0333e 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1182,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1204,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT if one of the subscriptions is
+ * not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1256,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1277,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1318,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7b4e8629553..5febd154b6b 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index c90f23ee5b0..accfa94badd 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index fd11805a44c..3c293b5a75b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -132,6 +132,94 @@
* failover = true when creating the subscription. Enabling failover allows us
* to smoothly transition to the promoted standby, ensuring that we can
* subscribe to the new primary without losing any data.
+ *
+ * RETAIN CONFLICT INFO
+ * ----------------------
+ * Each apply worker that enabled retain_conflict_info option maintains a
+ * non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
+ * prevent dead rows from being removed prematurely when the apply worker still
+ * needs them to detect conflicts reliably. This helps to retain the required
+ * commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * The logical replication launcher manages an internal replication slot named
+ * "pg_conflict_detection". It asynchronously aggregates the non-removable
+ * transaction ID from all apply workers to determine the appropriate xmin for
+ * the slot, thereby retaining necessary tuples.
+ *
+ * The non-removable transaction ID in the apply worker 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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.
+ *
+ * Similarly, when the publisher has subscribed to another publisher,
+ * information necessary for conflict detection cannot be retained for
+ * changes from origins other than the publisher. This is because the publisher
+ * lacks the transaction status of other publishers it subscribes to.
+ * Consequently, the non-removable transaction ID might be advanced prematurely
+ * before changes from other origins have been fully applied.
+ *
+ * XXX Retaining information for changes from other origins might be possible
+ * by requesting the subscription on that origin to enable retain_conflict_info
+ * and fetching the conflict detection slot.xmin along with the publisher's
+ * status. In the RCI_WAIT_FOR_PUBLISHER_STATUS phase, the apply worker could
+ * wait for the remote slot's xmin to reach the oldest active transaction ID,
+ * ensuring that all transactions from other origins have been applied on the
+ * publisher, thereby getting the latest WAL position that includes all
+ * concurrent changes. However, this approach may impact performance, so it
+ * might not worth the effort.
+ *
+ * XXX It seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
*-------------------------------------------------------------------------
*/
@@ -140,6 +228,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +255,14 @@
#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"
#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"
@@ -268,6 +359,78 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * See comments atop worker.c 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 */
+
+ /*
+ * Oldest transaction ID that was in the commit phase on the publisher.
+ * Use FullTransactionId to prevent issues with transaction ID wraparound,
+ * where a new remote_oldestxid could falsely appear to originate from the
+ * past and block advancement.
+ */
+ FullTransactionId remote_oldestxid;
+
+ /*
+ * Next transaction ID to be assigned on the publisher. Use
+ * FullTransactionId for consistency and to allow straightforward
+ * comparisons with remote_oldestxid.
+ */
+ FullTransactionId remote_nextxid;
+
+ TimestampTz reply_time; /* when the publisher responds with status */
+
+ /*
+ * Publisher transaction ID that must be awaited to complete before
+ * entering the final phase (RCI_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same reason as remote_nextxid.
+ */
+ FullTransactionId remote_wait_for;
+
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +495,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +542,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3760,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3839,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3867,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3884,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3921,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3960,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +4031,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +4068,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;
@@ -3920,6 +4145,363 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
last_flushpos = flushpos;
}
+/*
+ * Attempt to advance the non-removable transaction ID.
+ *
+ * See comments atop worker.c for details.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop worker.c for details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop worker.c.
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5290,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5470,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5840,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index f9fec50ae88..2017d399961 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index f2c33250e8b..466bef3e927 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 e5b945a9ee3..ce87f40e986 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2875,9 +2875,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2904,6 +2907,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2911,6 +2916,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 1937997ea67..19517c51a9b 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 30579ef2051..ae1a3262ad4 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -28,7 +28,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -631,7 +631,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();
}
@@ -764,7 +764,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -2040,48 +2040,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -2094,12 +2126,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2211,6 +2255,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..9436fb7c310 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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 e46f02c6cc6..11e7e21478a 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..dbcf88be945 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ 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 53e7d35fe98..3436092c0fd 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2306,8 +2306,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3744,8 +3745,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_proc.dat b/src/include/catalog/pg_proc.dat
index d4650947c63..f0943fa9b0a 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11793,6 +11793,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index ffacba9d2ae..9c8c7b9840b 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 ef0b733ebe8..e61d77534a0 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -56,7 +56,7 @@ extern bool TransactionIdIsInProgress(TransactionId xid);
extern bool TransactionIdIsActive(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 1443e1d9292..d5624c110fe 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,34 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | 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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..9b675b95407 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,195 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+##################################################
+# Check the WARNING when changing the origin to ANY, if retain_conflict_info is
+# enabled. This warns of the possibility of receiving changes from origins
+# other than the publisher.
+##################################################
+
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = any);");
+ok( $stderr =~
+ /WARNING: subscription "tap_sub_a_b" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins/,
+ "warn of the possibility of receiving changes from origins other than the publisher");
+
+# Reset the origin to none
+$node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = none);");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index 114bdafafdf..78efc9f4602 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2564,6 +2564,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.30.0.windows.2
On Mon, Jul 7, 2025 at 12:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Sun, Jul 6, 2025 at 10:51 PM Masahiko Sawada wrote:
On Sun, Jul 6, 2025 at 8:03 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:Dear hackers,
As a confirmation purpose, I did performance testing with four
workloads we did before.Thank you for doing the performance tests!
03. pgbench on both sides
========================
The workload is mostly same as [3].Workload:
- Ran pgbench with 40 clients for the *both side*.
- The duration was 120s, and the measurement was repeated 10 times.(bothtest.tar.gz can run the same workload)
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 16799.67659
- pgHead + patch : Median TPS = 17338.38423
Subscriber:
- pgHead : Median TPS = 16552.60515
- pgHead + patch : Median TPS = 8367.133693My first impression is that 40 clients is a small number at which a 50%
performance degradation occurs in 120s. Did you test how many clients are
required to trigger the same level performance regression with
retain_conflict_info = off?Could you please elaborate further on the intention behind the suggested tests
and what outcomes are expected? I ask because we anticipate that disabling
retain_conflict_info should not cause regression, given that dead tuples will
not be retained.
I think these performance regressions occur because at some point the
subscriber can no longer keep up with the changes occurring on the
publisher. This is because the publisher runs multiple transactions
simultaneously, while the Subscriber applies them with one apply
worker. When retain_conflict_info = on, the performance of the apply
worker deteriorates because it retains dead tuples, and as a result it
gradually cannot keep up with the publisher, the table bloats, and the
TPS of pgbench executed on the subscriber is also affected. This
happened when only 40 clients (or 15 clients according to the results
of test 4?) were running simultaneously.
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.
Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.
04. pgbench on both side, and max_conflict_retention_duration was
tuned================================================
========================
The workload is mostly same as [4].Workload:
- Initially ran pgbench with 40 clients for the *both side*.
- Set max_conflict_retention_duration = {60, 120}
- When the slot is invalidated on the subscriber side, stop the benchmarkand
wait until the subscriber would be caught up. Then the number of clients
on
the publisher would be half.
In this test the conflict slot could be invalidated as expected when theworkload
on the publisher was high, and it would not get invalidated anymore after
reducing the workload. This shows even if the slot has been invalidatedonce,
users can continue to detect the update_deleted conflict by reduce the
workload on the publisher.
- Total period of the test was 900s for each cases.(max_conflixt.tar.gz can run the same workload)
Observation:
-
- Parallelism of the publisher side is reduced till 15->7->3 and finally the
conflict slot is not invalidated.
- TPS on the subscriber side is improved when the concurrency wasreduced.
This is because the dead tuple accumulation is reduced on subscriber
due to
the reduced workload on the publisher.
- when publisher has Nclients=3, no regression in subscriber's TPSI think that users typically cannot control the amount of workloads in
production, meaning that once the performance regression starts to happen
the subscriber could enter the loop where invalidating the slot, recovreing the
performance, creating the slot, and having the performance problem.Yes, you are right. The test is designed to demonstrate that the slot can
be invalidated under high workload conditions as expected, while
remaining valid if the workload is reduced. In production systems where
workload reduction may not be possible, it’s recommended to disable
`retain_conflict_info` to enhance performance. This decision involves balancing
the need for reliable conflict detection with optimal system performance.
Agreed. I'm a bit concerned that the range in which users can achieve
this balance is small.
I think the hot standby feedback also has a similar impact on the performance
of the primary, which is done to prevent the early removal of data necessary
for the standby, ensuring that it remains accessible when needed.
Right. I think it's likely to happen if there is a long running
read-only query on the standby. But does it happen also when there are
only short read-only transactions on the standbys?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Mon, Jul 7, 2025 at 12:29 PM Hayato Kuroda (Fujitsu)
<kuroda.hayato@fujitsu.com> wrote:
Dear Sawada-san,
What does each duration mean in these results? Can we interpret the
test case of max_conflict_retention_duration=120s that when 7 clients
and 15 clients are working on the publisher and the subscriber
respectively, the TPS on the subscriber was about one fourth (17835.3
vs. 4707)?Firstly, this workload is done to prove that users can tune their workload to keep
enabling the update_deleted detections. Let me describe what happened there with
the timetable since the test starts.0-162s:
Number of clients on both publisher/subscriber was 15. TPS was 17835.3 on the
publisher and 4571.8 on the subscriber. This means that retained dead tuples on
the subscriber may reduce the performance to around 1/4 compared with publisher,
and the workload on the publisher is too heavy to keep working the update_deleted
detection.163-314s:
Number of clients was 7 on publisher, and 15 on subscriber. TPS was 9503.8 on
the publisher and 4707 on the subscriber. This means that N=7 on the publisher
was still too many thus conflict slot must be invalidated.315-597s:
Number of clients was 3 on publisher, and 15 on subscriber. TPS was 4243.9 on
the publisher and 19568.4 on the subscriber. Here the conflict slot could survive
during the benchmark because concurrency on the publisher was reduced.
Performance could be improved on the subscriber side because dead tuples can be
reduced here.
Thank you for your explanation!
Since this feature is designed to reliably detect conflicts on the
subscriber side, this scenario, where both the publisher and
subscriber are under load, represents a typical use case.
The fact that the subscriber can withstand the case where N=7 on the
publisher and N=15 on the subscriber with retain_conflict_info =
false, but fails to do so when retain_conflict_info = true, might
suggests a significant performance impact from enabling this feature.
In these test cases, was autovacuum disabled? I'm curious whether
users would experience permanently reduced transaction throughput, or
if this performance drop is temporary and recovers after autovacuum
runs.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Tue, Jul 8, 2025 at 12:18 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Jul 7, 2025 at 12:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I think these performance regressions occur because at some point the
subscriber can no longer keep up with the changes occurring on the
publisher. This is because the publisher runs multiple transactions
simultaneously, while the Subscriber applies them with one apply
worker. When retain_conflict_info = on, the performance of the apply
worker deteriorates because it retains dead tuples, and as a result it
gradually cannot keep up with the publisher, the table bloats, and the
TPS of pgbench executed on the subscriber is also affected. This
happened when only 40 clients (or 15 clients according to the results
of test 4?) were running simultaneously.
I think here the primary reason is the speed of one apply worker vs.
15 or 40 clients working on the publisher, and all the data is being
replicated. We don't see regression at 3 clients, which suggests apply
worker is able to keep up with that much workload. Now, we have
checked that if the workload is slightly different such that fewer
clients (say 1-3) work on same set of tables and then we make
different set of pub-sub pairs for all such different set of clients
(for example, 3 clients working on tables t1 and t2, other 3 clients
working on tables t3 and t4; then we can have 2 pub-sub pairs, one for
tables t1, t2, and other for t3-t4 ) then there is almost negligible
regression after enabling retain_conflict_info. Additionally, for very
large transactions that can be parallelized, we shouldn't see any
regression because those can be applied in parallel.
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.
As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]/messages/by-id/7b60e4e1-de40-4956-8135-cb1dc2be62e9@garret.ru. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.
I think the hot standby feedback also has a similar impact on the performance
of the primary, which is done to prevent the early removal of data necessary
for the standby, ensuring that it remains accessible when needed.Right. I think it's likely to happen if there is a long running
read-only query on the standby. But does it happen also when there are
only short read-only transactions on the standbys?
IIUC, the regression happens simply by increasing the value of
recovery_min_apply_delay. See case 5 in email [2]/messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com. This is to show the
point that we can see some regression in physical replication when
there is a delay in replication.
[1]: /messages/by-id/7b60e4e1-de40-4956-8135-cb1dc2be62e9@garret.ru
[2]: /messages/by-id/CABdArM4OEwmh_31dQ8_F__VmHwk2ag_M=YDD4H+yYQBG+bHGzg@mail.gmail.com
--
With Regards,
Amit Kapila.
Dear hackers,
All the tests in [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com are done with autovacuum=off, so I checked how would be in
autovacuum=on case.
Highlights
==========
* Regression on subscriber-side became bit larger than autovacuum=on case
when pgbench was run on both side
* Other than that, they were mostly same.
Used source
===========
pgHead commit fd7d7b7191 + v46 patchset
Machine details
===============
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
01. pgbench on publisher
========================
The workload is mostly same as [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com, but autovacuum was enabled.
Workload:
- Ran pgbench with 40 clients for the publisher.
- The duration was 300s, and the measurement was repeated 3 times.
Test Scenarios & Results:
- pgHead : Median TPS = 39757.24404
- pgHead + patch : Median TPS = 40871.37782
Observation:
- same trend as autovacuum=off
- No performance regression observed with the patch applied.
- The results were consistent across runs.
Detailed Results Table:
- each cell shows the TPS in each case.
- patch(ON) means patched and retain_conflict_info=ON is set.
run# pgHEAD pgHead+patch(ON)
1 40020.19703 40871.37782
2 39650.27381 40299.31806
3 39757.24404 41017.21089
median 39757.24404 40871.37782
02. pgbench on subscriber
========================
The workload is mostly same as [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com.
Workload:
- Ran pgbench with 40 clients for the *subscriber*.
- The duration was 300s, and the measurement was repeated 3 times.
Test Scenarios & Results:
- pgHead : Median TPS = 41552.27857
- pgHead + patch : Median TPS = 41677.02942
Observation:
- same trend as autovacuum=off
- No performance regression observed with the patch applied.
- The results were consistent across runs.
Detailed Results Table:
run# pgHEAD pgHead+patch(ON)
1 41656.71589 41673.42577
2 41552.27857 41677.02942
3 41504.98347 42114.66964
median 41552.27857 41677.02942
03. pgbench on both sides
========================
The workload is mostly same as [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com.
Workload:
- Ran pgbench with 15 clients for the *both side*.
- The duration was 300s, and the measurement was repeated 3 times.
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 17355.08998
- pgHead + patch : Median TPS = 18382.41213
Subscriber:
- pgHead : Median TPS = 16998.14496
- pgHead + patch : Median TPS = 5804.129821
Observation:
- Regression became larger than autovacuum = off (-50%->-60%)
- No performance regression observed on the publisher with the patch applied.
- The performance is reduced on the subscriber side (TPS reduction (~60%)) due
to dead tuple retention for the conflict detection
Detailed Results Table:
On publisher:
run# pgHEAD pgHead+patch(ON)
1 17537.52375 18382.41213
2 17355.08998 18408.0712
3 17286.78467 18119.77276
median 17355.08998 18382.41213
On subscriber:
run# pgHEAD pgHead+patch(ON)
1 17130.63876 5886.375748
2 16998.14496 5737.799408
3 16891.2713 5804.129821
median 16998.14496 5804.129821
[1]: /messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Wed, Jul 9, 2025 at 5:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 8, 2025 at 12:18 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Jul 7, 2025 at 12:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I think these performance regressions occur because at some point the
subscriber can no longer keep up with the changes occurring on the
publisher. This is because the publisher runs multiple transactions
simultaneously, while the Subscriber applies them with one apply
worker. When retain_conflict_info = on, the performance of the apply
worker deteriorates because it retains dead tuples, and as a result it
gradually cannot keep up with the publisher, the table bloats, and the
TPS of pgbench executed on the subscriber is also affected. This
happened when only 40 clients (or 15 clients according to the results
of test 4?) were running simultaneously.I think here the primary reason is the speed of one apply worker vs.
15 or 40 clients working on the publisher, and all the data is being
replicated. We don't see regression at 3 clients, which suggests apply
worker is able to keep up with that much workload. Now, we have
checked that if the workload is slightly different such that fewer
clients (say 1-3) work on same set of tables and then we make
different set of pub-sub pairs for all such different set of clients
(for example, 3 clients working on tables t1 and t2, other 3 clients
working on tables t3 and t4; then we can have 2 pub-sub pairs, one for
tables t1, t2, and other for t3-t4 ) then there is almost negligible
regression after enabling retain_conflict_info. Additionally, for very
large transactions that can be parallelized, we shouldn't see any
regression because those can be applied in parallel.
Yes, in test case-03 [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com, the performance drop(~50%) observed on the
subscriber side was primarily due to a single apply worker handling
changes from 40 concurrent clients on the publisher, which led to the
accumulation of dead tuples.
To validate this and simulate a more realistic workload, designed a
test as suggested above, where multiple clients update different
tables, and multiple subscriptions exist on the subscriber (one per
table set).
A custom pgbench script was created to run pgbench on the publisher,
with each client updating a unique set of tables. On the subscriber
side, created one subscription per set of tables. Each
publication-subscription pair handles a distinct table set.
Highlights
==========
- Two tests were done with two different workloads - 15 and 45
concurrent clients, respectively.
- No regression was observed when publisher changes were processed by
multiple apply workers on the subscriber.
Used source
===========
pgHead commit 62a17a92833 + v47 patch set
Machine details
===============
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
01. pgbench on both sides (with 15 clients)
=====================================
Setup:
- Publisher and Subscriber nodes are created with configurations:
autovacuum = false
shared_buffers = '30GB'
-- Also, worker and logical replication related parameters were
increased as per requirement (see attached scripts for details).
Workload:
- The publisher has 15 sets of pgbench tables: Each set includes four
tables: pgbench_accounts, pgbench_tellers, pgbench_branches, and
pgbench_history, named as:
pgbench_accounts_0, pgbench_tellers_0, ..., pgbench_accounts_14,
pgbench_tellers_14, etc.
- Ran pgbench with 15 clients for the *both side*.
-- On publisher, each client updates *only one* set of pgbench
tables: e.g., client '0' updates the pgbench_xx_0 tables, client '1'
updates pgbench_xx_1 tables, and so on.
-- On Subscriber, there exists one subscription per set of tables
of the publisher, i.e, there is one apply worker consuming changes
corresponding to each client. So, #subscriptions on subscriber(15) =
#clients on publisher(15).
- On subscriber, the default pgbench workload is also run with 15 clients.
- The duration was 5 minutes, and the measurement was repeated 3 times.
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 10386.93507
- pgHead + patch : Median TPS = 10187.0887 (TPS reduced ~2%)
Subscriber:
- pgHead : Median TPS = 10006.3903
- pgHead + patch : Median TPS = 9986.269682 (TPS reduced ~0.2%)
Observation:
- No performance regression was observed on either the publisher or
subscriber with the patch applied.
- The TPS drop was under 2% on both sides, within expected case to
case variation range.
Detailed Results Table:
On publisher:
#run pgHEAD pgHead+patch(ON)
1 10477.26438 10029.36155
2 10261.63429 10187.0887
3 10386.93507 10750.86231
median 10386.93507 10187.0887
On subscriber:
#run pgHEAD pgHead+patch(ON)
1 10261.63429 9813.114002
2 9962.914457 9986.269682
3 10006.3903 10580.13015
median 10006.3903 9986.269682
~~~~
02. pgbench on both sides (with 45 clients)
=====================================
Setup:
- same as case 01.
Workload:
- Publisher has the same 15 sets of pgbench tables as in case-01 and
3 clients will be updating one set of tables.
- Ran pgbench with 45 clients for the *both side*.
-- On publisher, each client updates *three* set of pgbench tables:
e.g., clients '0','15' and '30' update pgbench_xx_0 tables, clients
'1', '16', and '31' update pgbench_xx_1 tables, and so on.
-- On Subscriber, there exists one subscription per set of tables
of the publisher, i.e, there is one apply worker consuming changes
corresponding to *three* clients of the publisher.
- On subscriber, the default pgbench workload is also run with 45 clients.
- The duration was 5 minutes, and the measurement was repeated 3 times.
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 13845.7381
- pgHead + patch : Median TPS = 13553.682 (TPS reduced ~2%)
Subscriber:
- pgHead : Median TPS = 10080.54686
- pgHead + patch : Median TPS = 9908.304381 (TPS reduced ~1.7%)
Observation:
- No significant performance regression observed on either the
publisher or subscriber with the patch applied.
- The TPS drop was under 2% on both sides, within expected case to
case variation range.
Detailed Results Table:
On publisher:
#run pgHEAD pgHead+patch(ON)
1 14446.62404 13616.81375
2 12988.70504 13425.22938
3 13845.7381 13553.682
median 13845.7381 13553.682
On subscriber:
#run pgHEAD pgHead+patch(ON)
1 10505.47481 9908.304381
2 9963.119531 9843.280308
3 10080.54686 9987.983147
median 10080.54686 9908.304381
~~~~
The scripts used to perform above tests are attached.
[1]: /messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com
Dear hackers,
As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.
I verified row filter idea by benchmark and this was the valid approach.
Please see the below report.
Highlights
=======
- No regression was observed when publisher changes were processed by multiple
apply workers on the subscriber.
Used source
=========
pgHead commit 62a17a92833 + v47 patch set
Machine details
===========
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
Setup
====
- Publisher and Subscriber nodes are created with configurations:
autovacuum = false
shared_buffers = '30GB'
-- Also, worker and logical replication related parameters were increased
as needed (see attached scripts for details).
Workload
======
- The publisher has 4 pgbench tables: pgbench_pub_accounts, pgbench_pub_tellers,
pgbench_pub_branches, and pgbench_pub_history
- Also the publisher has 15 publications, say pub_[0..14]. Each publications could
publish tuples which PK % 15 was same as their suffic
- Ran pgbench with 15 clients for the *both side*.
- On subscriber, there were 15 subscribers which subscribed one of the publication
- On subscriber, the default pgbench workload is also run.
- The duration was 5 minutes, and the measurement was repeated 3 times.
Test Scenarios & Results:
Publisher:
- pgHead : Median TPS = 12201.92205
- pgHead + patch : Median TPS = 12368.58001
(TPS reduced ~1.5%)
Subscriber:
- pgHead : Median TPS = 11264.78483
- pgHead + patch : Median TPS = 11471.8107
(TPS reduced ~1.8%)
Observation:
- No performance regression was observed on either the publisher or subscriber
with the patch applied.
Detailed Results Table
======================
Publisher:
#run head patched
1 12201.92205 12368.58001
2 12263.03531 12410.21465
3 12171.24214 12330.47522
median 12201.92205 12368.58001
Subscriber:
#run head patched
1 11383.51717 11471.8107
2 11264.78483 11422.47011
3 11146.6676 11518.8403
median 11264.78483 11471.8107
Best regards,
Hayato Kuroda
FUJITSU LIMITED
Dear hackers,
-- Also, worker and logical replication related parameters were increased
as needed (see attached scripts for details).
Sorry, I forgot to attach scripts.
Best regards,
Hayato Kuroda
FUJITSU LIMITED
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 8, 2025 at 12:18 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Mon, Jul 7, 2025 at 12:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I think these performance regressions occur because at some point the
subscriber can no longer keep up with the changes occurring on the
publisher. This is because the publisher runs multiple transactions
simultaneously, while the Subscriber applies them with one apply
worker. When retain_conflict_info = on, the performance of the apply
worker deteriorates because it retains dead tuples, and as a result it
gradually cannot keep up with the publisher, the table bloats, and the
TPS of pgbench executed on the subscriber is also affected. This
happened when only 40 clients (or 15 clients according to the results
of test 4?) were running simultaneously.I think here the primary reason is the speed of one apply worker vs.
15 or 40 clients working on the publisher, and all the data is being
replicated. We don't see regression at 3 clients, which suggests apply
worker is able to keep up with that much workload. Now, we have
checked that if the workload is slightly different such that fewer
clients (say 1-3) work on same set of tables and then we make
different set of pub-sub pairs for all such different set of clients
(for example, 3 clients working on tables t1 and t2, other 3 clients
working on tables t3 and t4; then we can have 2 pub-sub pairs, one for
tables t1, t2, and other for t3-t4 ) then there is almost negligible
regression after enabling retain_conflict_info. Additionally, for very
large transactions that can be parallelized, we shouldn't see any
regression because those can be applied in parallel.
I find that it could make the system vulnerable to replication delays.
If the subscriber can't keep up even for a little while, it will enter
a negative loop. In order to avoid this, users have to reduce the
number of changes to each table set to a few clients, which may not be
user-friendly.
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.
While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.
I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1]/messages/by-id/7b60e4e1-de40-4956-8135-cb1dc2be62e9@garret.ru and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1]/messages/by-id/7b60e4e1-de40-4956-8135-cb1dc2be62e9@garret.ru, parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.
The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch). Do you see any
better alternative for moving forward?
[1]: /messages/by-id/7b60e4e1-de40-4956-8135-cb1dc2be62e9@garret.ru
--
With Regards,
Amit Kapila.
On Fri, Jul 11, 2025 at 4:28 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1] and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1], parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch). Do you see any
better alternative for moving forward?
I was just thinking about what are the most practical use cases where
a user would need multiple active writer nodes. Most applications
typically function well with a single active writer node. While it's
beneficial to have multiple nodes capable of writing for immediate
failover (e.g., if the current writer goes down), or they select a
primary writer via consensus algorithms like Raft/Paxos, I rarely
encounter use cases where users require multiple active writer nodes
for scaling write workloads. However, others may have different
perspectives on this.
One common use case for multiple active writer nodes is in
geographically distributed systems. Here, a dedicated writer in each
zone can significantly reduce write latency by sending writes to the
nearest zone.
In a multi-zone replication setup with an active writer in each zone
and data replicated across all zones, performance can be impacted by
factors like network latency. However, if such configurations are
implemented wisely and subscriptions are managed effectively, this
performance impact can be minimized.
IMHO, the same principle applies to this case when
‘retain_conflict_info’ is set to ON. If this setting is enabled, it
should only be used where absolutely essential. Additionally, the user
or DBA must carefully consider other factors. For instance, if they
use a single subscriber in each zone and subscribe to everything
across all zones, performance will significantly degrade. However, if
managed properly by subscribing only to data relevant to each zone and
using multiple subscribers for parallel apply of different
tables/partitions to reduce delay, it should work fine.
Anyway this is just my thought and others may think differently. And
I am open to hearing others thoughts as well.
--
Regards,
Dilip Kumar
Google
On Thu, Jul 17, 2025 at 9:56 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Jul 11, 2025 at 4:28 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1] and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1], parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch). Do you see any
better alternative for moving forward?I was just thinking about what are the most practical use cases where
a user would need multiple active writer nodes. Most applications
typically function well with a single active writer node. While it's
beneficial to have multiple nodes capable of writing for immediate
failover (e.g., if the current writer goes down), or they select a
primary writer via consensus algorithms like Raft/Paxos, I rarely
encounter use cases where users require multiple active writer nodes
for scaling write workloads.
Thank you for the feedback. In the scenario with a single writer node
and a subscriber with RCI enabled, we have not observed any
regression. Please refer to the test report at [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com, specifically test
cases 1 and 2, which involve a single writer node. Next, we can test a
scenario with multiple (2-3) writer nodes publishing changes, and a
subscriber node subscribing to those writers with RCI enabled, which
can even serve as a good use case of the conflict detection we are
targeting through RCI enabling.
One common use case for multiple active writer nodes is in
geographically distributed systems. Here, a dedicated writer in each
zone can significantly reduce write latency by sending writes to the
nearest zone.In a multi-zone replication setup with an active writer in each zone
and data replicated across all zones, performance can be impacted by
factors like network latency. However, if such configurations are
implemented wisely and subscriptions are managed effectively, this
performance impact can be minimized.IMHO, the same principle applies to this case when
‘retain_conflict_info’ is set to ON. If this setting is enabled, it
should only be used where absolutely essential. Additionally, the user
or DBA must carefully consider other factors. For instance, if they
use a single subscriber in each zone and subscribe to everything
across all zones, performance will significantly degrade. However, if
managed properly by subscribing only to data relevant to each zone and
using multiple subscribers for parallel apply of different
tables/partitions to reduce delay, it should work fine.
Strongly agree with this. We tested scenarios involving multiple
subscribers, each subscribing to exclusive data, as well as
publishers using row filters. In both cases, no regressions were
observed. Please refer to the test results at [2]row filter - /messages/by-id/OSCPR01MB149660DD40A9D7C18E2E11C97F548A@OSCPR01MB14966.jpnprd01.prod.outlook.com and [3]Multiple subscriptions - /messages/by-id/CABdArM5kvA7mPLLwy6XEDkHi0MNs1RidvAcYmm2uVd95U=yzwQ@mail.gmail.com.
[1]: /messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com
/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com
[2]: row filter - /messages/by-id/OSCPR01MB149660DD40A9D7C18E2E11C97F548A@OSCPR01MB14966.jpnprd01.prod.outlook.com
row filter - /messages/by-id/OSCPR01MB149660DD40A9D7C18E2E11C97F548A@OSCPR01MB14966.jpnprd01.prod.outlook.com
[3]: Multiple subscriptions - /messages/by-id/CABdArM5kvA7mPLLwy6XEDkHi0MNs1RidvAcYmm2uVd95U=yzwQ@mail.gmail.com
Multiple subscriptions -
/messages/by-id/CABdArM5kvA7mPLLwy6XEDkHi0MNs1RidvAcYmm2uVd95U=yzwQ@mail.gmail.com
thanks
Shveta
On Fri, Jul 11, 2025 at 3:58 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1] and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1], parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.
I don't have a better alternative either.
I agree that this feature will work without any problem when logical
replication is properly configured. It's a good point that
update-delete conflicts can be detected reliably without additional
performance overhead in scenarios with minimal replication lag.
However, this approach requires users to carefully pay particular
attention to replication performance and potential delays. My primary
concern is that, given the current logical replication performance
limitations, most users who want to use this feature will likely need
such dedicated care for replication lag. Nevertheless, most features
involve certain trade-offs. Given that this is an opt-in feature and
future performance improvements will reduce these challenges for
users, it would be reasonable to have this feature at this stage.
The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch).
+1
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Thu, Jul 17, 2025 at 4:44 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Jul 17, 2025 at 9:56 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
I was just thinking about what are the most practical use cases where
a user would need multiple active writer nodes. Most applications
typically function well with a single active writer node. While it's
beneficial to have multiple nodes capable of writing for immediate
failover (e.g., if the current writer goes down), or they select a
primary writer via consensus algorithms like Raft/Paxos, I rarely
encounter use cases where users require multiple active writer nodes
for scaling write workloads.Thank you for the feedback. In the scenario with a single writer node
and a subscriber with RCI enabled, we have not observed any
regression. Please refer to the test report at [1], specifically test
cases 1 and 2, which involve a single writer node. Next, we can test a
scenario with multiple (2-3) writer nodes publishing changes, and a
subscriber node subscribing to those writers with RCI enabled, which
can even serve as a good use case of the conflict detection we are
targeting through RCI enabling.
+1
One common use case for multiple active writer nodes is in
geographically distributed systems. Here, a dedicated writer in each
zone can significantly reduce write latency by sending writes to the
nearest zone.In a multi-zone replication setup with an active writer in each zone
and data replicated across all zones, performance can be impacted by
factors like network latency. However, if such configurations are
implemented wisely and subscriptions are managed effectively, this
performance impact can be minimized.IMHO, the same principle applies to this case when
‘retain_conflict_info’ is set to ON. If this setting is enabled, it
should only be used where absolutely essential. Additionally, the user
or DBA must carefully consider other factors. For instance, if they
use a single subscriber in each zone and subscribe to everything
across all zones, performance will significantly degrade. However, if
managed properly by subscribing only to data relevant to each zone and
using multiple subscribers for parallel apply of different
tables/partitions to reduce delay, it should work fine.Strongly agree with this. We tested scenarios involving multiple
subscribers, each subscribing to exclusive data, as well as
publishers using row filters. In both cases, no regressions were
observed. Please refer to the test results at [2] and [3].
Right, thanks for pointing me to the related test cases.
--
Regards,
Dilip Kumar
Google
On Thu, Jul 17, 2025 at 4:44 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Jul 17, 2025 at 9:56 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Jul 11, 2025 at 4:28 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
I think that even with retain_conflict_info = off, there is probably a
point at which the subscriber can no longer keep up with the
publisher. For example, if with retain_conflict_info = off we can
withstand 100 clients running at the same time, then the fact that
this performance degradation occurred with 15 clients explains that
performance degradation is much more likely to occur because of
retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used since
the conflicts actually happen on the subscriber, so I think it's
important to look at the performance in these cases. The worst case
scenario for this feature is that when this feature is turned on, the
subscriber cannot keep up even with a small load, and with
max_conflict_retetion_duration we enter a loop of slot invalidation
and re-creating, which means that conflict cannot be detected
reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of pre-fetch
for apply, as is recently proposed [1]. I feel there are use cases, as
explained above, for which this feature would work without any
downside, but due to a lack of some sort of parallel apply, we may not
be able to use it without any downside for cases where the contention
is only on a smaller set of tables. We have not tried, but may in
cases where contention is on a smaller set of tables, if users
distribute workload among different pub-sub pairs by using row
filters, there also, we may also see less regression. We can try that
as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1] and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1], parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch). Do you see any
better alternative for moving forward?I was just thinking about what are the most practical use cases where
a user would need multiple active writer nodes. Most applications
typically function well with a single active writer node. While it's
beneficial to have multiple nodes capable of writing for immediate
failover (e.g., if the current writer goes down), or they select a
primary writer via consensus algorithms like Raft/Paxos, I rarely
encounter use cases where users require multiple active writer nodes
for scaling write workloads.Thank you for the feedback. In the scenario with a single writer node
and a subscriber with RCI enabled, we have not observed any
regression. Please refer to the test report at [1], specifically test
cases 1 and 2, which involve a single writer node. Next, we can test a
scenario with multiple (2-3) writer nodes publishing changes, and a
subscriber node subscribing to those writers with RCI enabled, which
can even serve as a good use case of the conflict detection we are
targeting through RCI enabling.
I did a workload test for the setup as suggested above - "we can test
a scenario with multiple (2-3) writer nodes publishing changes, and a
subscriber node subscribing to those writers with RCI enabled".
Here are the results :
Highlights
==========
- Two tests were done with two different workloads - 15 and 40
concurrent clients, respectively.
- No regression was observed on any of the nodes.
Used source
===========
pgHead commit 62a17a92833 + v47 patch set
Machine details
===============
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
01. pgbench with 15 clients
========================
Setup:
- Two publishers and one subscriber:
pub1 --> sub
pub2 --> sub
- All three nodes have same pgbench tables (scale=60) and are configured with:
autovacuum = false
shared_buffers = '30GB'
-- Also, worker and logical replication related parameters were
increased as per requirement (see attached scripts for details).
- The topology is such that pub1 & pub2 are independent writers. The
sub acts as reader(no writes) and has subscribed for all the changes
from both pub1 and pub2.
Workload:
- pgbench (read-write) was run on both pub1 and pub2 (15 clients,
duration = 5 minutes)
- pgbench (read-only) was run on sub (15 clients, duration = 5 minutes)
- The measurement was repeated 2 times.
Observation:
- No performance regression was observed on either the writer nodes
(publishers) or the reader node (subscriber) with the patch applied.
- TPS on both publishers was slightly better than on pgHead. This
could be because all nodes run on the same machine - under high
publisher load, the subscriber's apply worker performs I/O more slowly
due to dead tuple retention, giving publisher-side pgbench more I/O
bandwidth to complete writes. We can investigate further if needed.
Detailed Results Table:
On publishers:
#run pgHead_Pub1_TPS pgHead_Pub2_TPS patched_pub1_TPS patched_pub2_TPS
1 13440.47394 13459.71296 14325.81026 14345.34077
2 13529.29649 13553.65741 14382.32144 14332.94777
median 13484.88521 13506.68518 14354.06585 14339.14427
- No regression
On subscriber:
#run pgHead_sub_TPS patched_sub_TPS
1 127009.0631 126894.9649
2 127767.4083 127207.8632
median 127388.2357 127051.4141
- No regression
~~~~
02. pgbench with 40 clients
======================
Setup:
- same as case-01
Workload:
- pgbench (read-write) was run on both pub1 and pub2 (40 clients,
duration = 10 minutes)
- pgbench (read-only) was run on sub (40 clients, duration = 10 minutes)
- The measurement was repeated 2 times.
Observation:
- No performance regression was observed on any writer nodes, i.e,
the publishers, or the reader node i.e., subscriber with the patch
applied.
- Similar to case-01, TPS on both publishers was slightly higher than
on pgHead.
Detailed Results Table:
On publisher:
#run pgHead_Pub1_TPS patched_pub1_TPS pgHead_Pub2_TPS patched_pub2_TPS
1 17818.12479 18602.42504 17744.77163 18620.90056
2 17759.3144 18660.44407 17774.47442 18230.63849
median 17788.7196 18631.43455 17759.62302 18425.76952
- No regression
On subscriber:
#run pgHead_sub_TPS patched_sub_TPS
1 281075.3732 279438.4882
2 275988.1383 277388.6316
median 278531.7557 278413.5599
- No regression
~~~~
The scripts used to perform above tests are attached.
--
Thanks,
Nisha
On Friday, July 18, 2025 1:25 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jul 11, 2025 at 3:58 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
I think that even with retain_conflict_info = off, there is
probably a point at which the subscriber can no longer keep up
with the publisher. For example, if with retain_conflict_info =
off we can withstand 100 clients running at the same time, then
the fact that this performance degradation occurred with 15
clients explains that performance degradation is much more
likely to occur because of retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used
since the conflicts actually happen on the subscriber, so I
think it's important to look at the performance in these cases.
The worst case scenario for this feature is that when this
feature is turned on, the subscriber cannot keep up even with a
small load, and with max_conflict_retetion_duration we enter a
loop of slot invalidation and re-creating, which means that
conflict cannot be detected reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of
pre-fetch for apply, as is recently proposed [1]. I feel there are
use cases, as explained above, for which this feature would work
without any downside, but due to a lack of some sort of parallel
apply, we may not be able to use it without any downside for cases
where the contention is only on a smaller set of tables. We have
not tried, but may in cases where contention is on a smaller set
of tables, if users distribute workload among different pub-sub
pairs by using row filters, there also, we may also see less
regression. We can try that as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1] and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1], parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.I don't have a better alternative either.
I agree that this feature will work without any problem when logical replication
is properly configured. It's a good point that update-delete conflicts can be
detected reliably without additional performance overhead in scenarios with
minimal replication lag.
However, this approach requires users to carefully pay particular attention to
replication performance and potential delays. My primary concern is that, given
the current logical replication performance limitations, most users who want to
use this feature will likely need such dedicated care for replication lag.
Nevertheless, most features involve certain trade-offs. Given that this is an
opt-in feature and future performance improvements will reduce these
challenges for users, it would be reasonable to have this feature at this stage.The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch).+1
Thanks for the discussion. Here is the V49 patch which includes the suggested
doc change in 0002. I will rebase the remaining patches once the first one is
pushed.
Thanks to Shveta for preparing the doc change.
Best Regards,
Hou zj
Attachments:
v49-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v49-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From c89cf2170d68de2f5c7114feea761777fbc9e62c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 18 Jul 2025 17:31:20 +0800
Subject: [PATCH v49] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_conflict_info option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_conflict_info enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 50 +-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 357 ++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 221 +++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 666 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 2 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/logicalworker.h | 5 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 +++--
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 195 ++++-
src/tools/pgindent/typedefs.list | 2 +
45 files changed, 2180 insertions(+), 215 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 0d23bc1b122..79ccd35a208 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretainconflictinfo</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index c7acc0f182f..20ccb2d6b54 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4965,6 +4965,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index f5a0e0954a1..de5b5929ee0 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29592,7 +29592,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29636,7 +29638,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29666,6 +29670,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29688,8 +29694,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index e26f7f59d4a..b2a4e74b079 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2396,6 +2396,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2532,6 +2538,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2563,6 +2585,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_conflict_info enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index e74b5be1eff..b115884acb3 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2235,6 +2235,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2653,6 +2655,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2797,6 +2858,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..da9b559f18e 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-conflict-info"><literal>retain_conflict_info</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ conflict information, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..7782246727e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,52 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-conflict-info">
+ <term><literal>retain_conflict_info</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_conflict_info</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+
+ <para>
+ Enabling this option ensures retention of information useful for
+ conflict detection solely for changes occurring locally on the
+ publisher. For the changes originating from different origins,
+ reliable conflict detection cannot be guaranteed.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 85cbe397cb2..7c94d020d7a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1183,7 +1183,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 in logical replication. See
+ * GetOldestActiveTransactionId(true) and its use.
*
* We save the PREPARE record's location in the gxact for later use by
* CheckPointTwoPhase.
@@ -2298,7 +2302,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2318,7 +2322,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2331,8 +2335,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 comments atop worker.c.
+ */
+ committs = GetCurrentTimestamp();
/*
* Emit the XLOG commit record. Note that we mark 2PC commits as
@@ -2381,7 +2401,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 41601fcb280..b46e7e9c2a6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 8e7827c6ed9..ed7d194ab7f 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7121,7 +7121,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 23878b2dd91..e8f3ba00caa 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..39cfae43d6f 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 b2d5332effc..a9b008beef0 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1386,7 +1386,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 e23b0de7242..97914dfe53b 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;
@@ -105,8 +108,11 @@ typedef struct SubOpts
static List *fetch_table_list(WalReceiverConn *wrconn, List *publications);
static void check_publications_origin(WalReceiverConn *wrconn,
List *publications, bool copydata,
- char *origin, Oid *subrel_local_oids,
- int subrel_count, char *subname);
+ bool retain_conflict_info, char *origin,
+ Oid *subrel_local_oids, int subrel_count,
+ char *subname);
+static void check_pub_conflict_info_retention(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 +168,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);
@@ -210,7 +218,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)
@@ -307,6 +315,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 +580,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);
/*
@@ -630,6 +648,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retainconflictinfo. */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo, true,
+ !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +692,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)
@@ -722,7 +746,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
{
check_publications(wrconn, publications);
check_publications_origin(wrconn, publications, opts.copy_data,
- opts.origin, NULL, 0, stmt->subname);
+ opts.retainconflictinfo, opts.origin,
+ NULL, 0, stmt->subname);
+
+ check_pub_conflict_info_retention(wrconn, opts.retainconflictinfo);
/*
* Set sync state based on if we were asked to do data copy or
@@ -881,8 +908,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
sizeof(Oid), oid_cmp);
check_publications_origin(wrconn, sub->publications, copy_data,
- sub->origin, subrel_local_oids,
- subrel_count, sub->name);
+ sub->retainconflictinfo, sub->origin,
+ subrel_local_oids, subrel_count, sub->name);
/*
* Rels that we want to remove from subscription and drop any slots
@@ -1040,18 +1067,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_conflict_info
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_conflict_info") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_conflict_info option does not update the slot on
+ * the publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_conflict_info") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1094,41 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_conflict_info option
+ * when the subscription is enabled to prevent race conditions arising
+ * from the new option value being acknowledged asynchronously by the
+ * launcher and apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_conflict_info option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_conflict_info concurrently with
+ * the launcher starting the worker, the apply worker may start
+ * calculating oldest_nonremovable_xid before the launcher notices the
+ * enable action. Consequently, the launcher may update slot.xmin to a
+ * newer value than that maintained by the worker. In subsequent cycles,
+ * upon integrating the worker's oldest_nonremovable_xid, the launcher
+ * might detect a retreat in the calculated xmin, necessitating additional
+ * handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the
+ * retain_conflict_info, but in the future we can change this after some
+ * more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_conflict_info
+ * to avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1176,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_tuple = false;
bool update_failover = false;
bool update_two_phase = false;
+ bool check_pub_rci = false;
+ bool retain_conflict_info;
+ char *origin;
Subscription *sub;
Form_pg_subscription form;
bits32 supported_opts;
@@ -1137,6 +1206,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
sub = GetSubscription(subid, false);
+ retain_conflict_info = sub->retainconflictinfo;
+ origin = sub->origin;
+
/*
* Don't allow non-superuser modification of a subscription with
* password_required=false.
@@ -1165,7 +1237,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,11 +1397,62 @@ 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;
+
+ CheckAlterSubOption(sub, "retain_conflict_info", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_conflict_info when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ CheckSubConflictInfoRetention(opts.retainconflictinfo,
+ true, !sub->enabled, NOTICE);
+
+ /*
+ * 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();
+
+ check_pub_rci = opts.retainconflictinfo;
+ retain_conflict_info = opts.retainconflictinfo;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
CStringGetTextDatum(opts.origin);
replaces[Anum_pg_subscription_suborigin - 1] = true;
+
+ /*
+ * Check if changes from different origins may be received
+ * from the publisher when the origin is changed to ANY
+ * and retain_conflict_info is enabled.
+ */
+ check_pub_rci = retain_conflict_info &&
+ pg_strcasecmp(opts.origin, LOGICALREP_ORIGIN_ANY) == 0;
+
+ origin = opts.origin;
}
update_tuple = true;
@@ -1347,6 +1470,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubConflictInfoRetention() for details.
+ */
+ CheckSubConflictInfoRetention(sub->retainconflictinfo,
+ opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1487,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.
+ */
+ check_pub_rci = sub->retainconflictinfo && opts.enabled;
break;
}
@@ -1369,6 +1509,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.
+ */
+ check_pub_rci = sub->retainconflictinfo;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1715,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 || check_pub_rci)
{
bool must_use_password;
char *err;
@@ -1584,10 +1732,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 +1748,16 @@ 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_pub_conflict_info_retention(wrconn, retain_conflict_info);
+
+ check_publications_origin(wrconn, sub->publications, false,
+ retain_conflict_info, origin, NULL, 0,
+ sub->name);
+
+ 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();
{
@@ -2086,20 +2245,29 @@ AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId)
* Check and log a warning if the publisher has subscribed to the same table,
* its partition ancestors (if it's a partition), or its partition children (if
* it's a partitioned table), from some other publishers. This check is
- * required only if "copy_data = true" and "origin = none" for CREATE
- * SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements to notify the
- * user that data having origin might have been copied.
+ * required in the following scenarios:
*
- * This check need not be performed on the tables that are already added
- * because incremental sync for those tables will happen through WAL and the
- * origin of the data can be identified from the WAL records.
+ * 1) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "copy_data = true" and "origin = none":
+ * - Warn the user that data with an origin might have been copied.
+ * - This check is skipped for tables already added, as incremental sync via
+ * WAL allows origin tracking. The list of such tables is in
+ * subrel_local_oids.
*
- * subrel_local_oids contains the list of relation oids that are already
- * present on the subscriber.
+ * 2) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "retain_conflict_info = true" and "origin = any", and for ALTER
+ * SUBSCRIPTION statements that modify retain_conflict_info or origin, or
+ * when the publisher's status changes (e.g., due to a connection string
+ * update):
+ * - Warn the user that only conflict detection info for local changes on
+ * the publisher is retained. Data from other origins may lack sufficient
+ * details for reliable conflict detection.
+ * - See comments atop worker.c for more details.
*/
static void
check_publications_origin(WalReceiverConn *wrconn, List *publications,
- bool copydata, char *origin, Oid *subrel_local_oids,
+ bool copydata, bool retain_conflict_info,
+ char *origin, Oid *subrel_local_oids,
int subrel_count, char *subname)
{
WalRcvExecResult *res;
@@ -2108,9 +2276,29 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
Oid tableRow[1] = {TEXTOID};
List *publist = NIL;
int i;
+ bool check_rci;
+ bool check_table_sync;
+ bool origin_none = origin &&
+ pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) == 0;
+
+ /*
+ * Enable retain_conflict_info checks only when origin is set to 'any',
+ * since with origin='none' only local changes are replicated to the
+ * subscriber.
+ */
+ check_rci = retain_conflict_info && !origin_none;
- if (!copydata || !origin ||
- (pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) != 0))
+ /*
+ * Enable table synchronization checks only when origin is 'none', to
+ * ensure that data from other origins is not inadvertently copied.
+ */
+ check_table_sync = copydata && origin_none;
+
+ /* retain_conflict_info and table sync checks occur separately */
+ Assert(!(check_rci && check_table_sync));
+
+ /* Return if no checks are required */
+ if (!check_rci && !check_table_sync)
return;
initStringInfo(&cmd);
@@ -2129,16 +2317,23 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
/*
* In case of ALTER SUBSCRIPTION ... REFRESH, subrel_local_oids contains
* the list of relation oids that are already present on the subscriber.
- * This check should be skipped for these tables.
+ * This check should be skipped for these tables if checking for table
+ * sync scenario. However, when handling the retain_conflict_info
+ * scenario, ensure all tables are checked, as some existing tables may
+ * now include changes from other origins due to newly created
+ * subscriptions on the publisher.
*/
- for (i = 0; i < subrel_count; i++)
+ if (check_table_sync)
{
- Oid relid = subrel_local_oids[i];
- char *schemaname = get_namespace_name(get_rel_namespace(relid));
- char *tablename = get_rel_name(relid);
+ for (i = 0; i < subrel_count; i++)
+ {
+ Oid relid = subrel_local_oids[i];
+ char *schemaname = get_namespace_name(get_rel_namespace(relid));
+ char *tablename = get_rel_name(relid);
- appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
- schemaname, tablename);
+ appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
+ schemaname, tablename);
+ }
}
res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
@@ -2173,22 +2368,37 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
* XXX: For simplicity, we don't check whether the table has any data or
* not. If the table doesn't have any data then we don't need to
* distinguish between data having origin and data not having origin so we
- * can avoid logging a warning in that case.
+ * can avoid logging a warning for table sync scenario.
*/
if (publist)
{
StringInfo pubnames = makeStringInfo();
+ StringInfo err_msg = makeStringInfo();
+ StringInfo err_hint = makeStringInfo();
/* Prepare the list of publication(s) for warning message. */
GetPublicationsStr(publist, pubnames, false);
+
+ if (check_table_sync)
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin"),
+ subname);
+ appendStringInfoString(err_hint, _("Verify that initial data copied from the publisher tables did not come from other origins."));
+ }
+ else
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins"),
+ subname);
+ appendStringInfoString(err_hint, _("Consider using origin = NONE or disabling retain_conflict_info."));
+ }
+
ereport(WARNING,
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin",
- subname),
- errdetail_plural("The subscription being created subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
- "The subscription being created subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
+ errmsg_internal("%s", err_msg->data),
+ errdetail_plural("The subscription subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
+ "The subscription subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
list_length(publist), pubnames->data),
- errhint("Verify that initial data copied from the publisher tables did not come from other origins."));
+ errhint_internal("%s", err_hint->data));
}
ExecDropSingleTupleTableSlot(slot);
@@ -2196,6 +2406,57 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_conflict_info can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop worker.c for a detailed explanation.
+ */
+static void
+check_pub_conflict_info_retention(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) < 19000)
+ 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 19"));
+
+ 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/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..f18fee0333e 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,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info,
+ TransactionId *xmin,
+ bool *can_advance_xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +154,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);
@@ -309,7 +316,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_conflict_info)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +336,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_conflict_info is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_conflict_info || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +465,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_conflict_info
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1132,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1167,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1182,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_conflict_info = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1178,13 +1204,33 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ retain_conflict_info |= sub->retainconflictinfo;
+
if (!sub->enabled)
+ {
+ /*
+ * This is required to ensure that we don't advance the xmin
+ * of CONFLICT_DETECTION_SLOT if one of the subscriptions is
+ * not enabled. Otherwise, we won't be able to detect
+ * conflicts reliably for such a subscription even though it
+ * has set the retain_conflict_info option.
+ */
+ compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
+ &xmin, &can_advance_xmin);
continue;
+ }
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection.
+ */
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
+ &can_advance_xmin);
+
if (w != NULL)
continue; /* worker is running already */
@@ -1210,7 +1256,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retainconflictinfo))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1277,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain the conflict information. Otherwise, if
+ * required, advance the slot's xmin to protect deleted tuples
+ * required for the conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_conflict_info)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1318,158 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_conflict_info enabled. Store the result
+ * in *xmin.
+ *
+ * If the replication slot cannot be advanced during this cycle, due to either
+ * a disabled subscription or an inactive worker, set *can_advance_xmin to
+ * false.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool retain_conflict_info, TransactionId *xmin,
+ bool *can_advance_xmin)
+{
+ if (!retain_conflict_info || !*can_advance_xmin)
+ return;
+
+ if (worker)
+ {
+ TransactionId nonremovable_xid;
+
+ /*
+ * The replication slot for conflict detection must be created before
+ * the worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+ }
+ else
+ {
+ /*
+ * Create a replication slot to retain information necessary for
+ * conflict detection such as dead tuples, commit timestamps, and
+ * origins.
+ *
+ * The slot is created before starting the apply worker to prevent it
+ * from unnecessarily maintaining its oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to ensure that
+ * conflict-related information is available when applying remote
+ * changes that occurred before the subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions with
+ * retain_conflict_info are enabled and the corresponding workers are
+ * running.
+ */
+ *can_advance_xmin = false;
+ }
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7b4e8629553..5febd154b6b 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index e4fd6347fd1..3fea0a0206e 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c5fb627aa56..55ce75d251e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -132,6 +132,94 @@
* failover = true when creating the subscription. Enabling failover allows us
* to smoothly transition to the promoted standby, ensuring that we can
* subscribe to the new primary without losing any data.
+ *
+ * RETAIN CONFLICT INFO
+ * ----------------------
+ * Each apply worker that enabled retain_conflict_info option maintains a
+ * non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
+ * prevent dead rows from being removed prematurely when the apply worker still
+ * needs them to detect conflicts reliably. This helps to retain the required
+ * commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * The logical replication launcher manages an internal replication slot named
+ * "pg_conflict_detection". It asynchronously aggregates the non-removable
+ * transaction ID from all apply workers to determine the appropriate xmin for
+ * the slot, thereby retaining necessary tuples.
+ *
+ * The non-removable transaction ID in the apply worker 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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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.
+ *
+ * Similarly, when the publisher has subscribed to another publisher,
+ * information necessary for conflict detection cannot be retained for
+ * changes from origins other than the publisher. This is because the publisher
+ * lacks the transaction status of other publishers it subscribes to.
+ * Consequently, the non-removable transaction ID might be advanced prematurely
+ * before changes from other origins have been fully applied.
+ *
+ * XXX Retaining information for changes from other origins might be possible
+ * by requesting the subscription on that origin to enable retain_conflict_info
+ * and fetching the conflict detection slot.xmin along with the publisher's
+ * status. In the RCI_WAIT_FOR_PUBLISHER_STATUS phase, the apply worker could
+ * wait for the remote slot's xmin to reach the oldest active transaction ID,
+ * ensuring that all transactions from other origins have been applied on the
+ * publisher, thereby getting the latest WAL position that includes all
+ * concurrent changes. However, this approach may impact performance, so it
+ * might not worth the effort.
+ *
+ * XXX It seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
*-------------------------------------------------------------------------
*/
@@ -140,6 +228,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -166,12 +255,14 @@
#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"
#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"
@@ -268,6 +359,78 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * See comments atop worker.c 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 */
+
+ /*
+ * Oldest transaction ID that was in the commit phase on the publisher.
+ * Use FullTransactionId to prevent issues with transaction ID wraparound,
+ * where a new remote_oldestxid could falsely appear to originate from the
+ * past and block advancement.
+ */
+ FullTransactionId remote_oldestxid;
+
+ /*
+ * Next transaction ID to be assigned on the publisher. Use
+ * FullTransactionId for consistency and to allow straightforward
+ * comparisons with remote_oldestxid.
+ */
+ FullTransactionId remote_nextxid;
+
+ TimestampTz reply_time; /* when the publisher responds with status */
+
+ /*
+ * Publisher transaction ID that must be awaited to complete before
+ * entering the final phase (RCI_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same reason as remote_nextxid.
+ */
+ FullTransactionId remote_wait_for;
+
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ 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 =
{
@@ -332,6 +495,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +542,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainConflictInfoData *rci_data);
+static void process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void get_candidate_xid(RetainConflictInfoData *rci_data);
+static void request_publisher_status(RetainConflictInfoData *rci_data);
+static void wait_for_publisher_status(RetainConflictInfoData *rci_data,
+ bool status_received);
+static void wait_for_local_flush(RetainConflictInfoData *rci_data);
+static void adjust_xid_advance_interval(RetainConflictInfoData *rci_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3760,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainConflictInfoData rci_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3839,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rci_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3867,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3884,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rci_data.remote_lsn = pq_getmsgint64(&s);
+ rci_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rci_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(rci_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rci_data, true);
+
+ UpdateWorkerStats(last_received, rci_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3921,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rci_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3960,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rci_data.phase == RCI_GET_CANDIDATE_XID &&
+ rci_data.xid_advance_interval)
+ wait_time = Min(wait_time, rci_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +4031,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rci_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +4068,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;
@@ -3920,6 +4145,363 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
last_flushpos = flushpos;
}
+/*
+ * Attempt to advance the non-removable transaction ID.
+ *
+ * See comments atop worker.c for details.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rci_data))
+ return;
+
+ process_rci_phase_transition(rci_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainConflictInfoData *rci_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining conflict information is not required */
+ if (!MySubscription->retainconflictinfo)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop worker.c for details of the transition.
+ */
+static void
+process_rci_phase_transition(RetainConflictInfoData *rci_data,
+ bool status_received)
+{
+ switch (rci_data->phase)
+ {
+ case RCI_GET_CANDIDATE_XID:
+ get_candidate_xid(rci_data);
+ break;
+ case RCI_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rci_data);
+ break;
+ case RCI_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rci_data, status_received);
+ break;
+ case RCI_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rci_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RCI_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainConflictInfoData *rci_data)
+{
+ TransactionId oldest_running_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 = rci_data->last_recv_time ? rci_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(rci_data->candidate_xid_time, now,
+ rci_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rci_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rci_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rci_data, true);
+
+ rci_data->candidate_xid = oldest_running_xid;
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainConflictInfoData *rci_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);
+
+ rci_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 *rci_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rci_data->remote_wait_for))
+ rci_data->remote_wait_for = rci_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RCI_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rci_data->remote_wait_for,
+ rci_data->remote_oldestxid))
+ rci_data->phase = RCI_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rci_data->phase = RCI_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_data, false);
+}
+
+/*
+ * Workhorse for the RCI_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainConflictInfoData *rci_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rci_data->remote_lsn) &&
+ TransactionIdIsValid(rci_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop worker.c.
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rci_data->reply_time,
+ rci_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rci_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rci_data->remote_lsn && rci_data->last_recv_time &&
+ TimestampDifferenceExceeds(rci_data->flushpos_update_time,
+ rci_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;
+
+ rci_data->flushpos_update_time = rci_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rci_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 = rci_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rci_data->remote_lsn),
+ rci_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rci_data->phase = RCI_GET_CANDIDATE_XID;
+ rci_data->remote_lsn = InvalidXLogRecPtr;
+ rci_data->remote_oldestxid = InvalidFullTransactionId;
+ rci_data->remote_nextxid = InvalidFullTransactionId;
+ rci_data->reply_time = 0;
+ rci_data->remote_wait_for = InvalidFullTransactionId;
+ rci_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rci_phase_transition(rci_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 *rci_data, bool new_xid_found)
+{
+ if (!new_xid_found && rci_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.
+ */
+ rci_data->xid_advance_interval = Min(rci_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.
+ */
+ rci_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5290,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_conflict_info was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_conflict_info as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_conflict_info is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retainconflictinfo &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_conflict_info"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5470,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ CheckSubConflictInfoRetention(MySubscription->retainconflictinfo, false,
+ true, WARNING);
+
proc_exit(0);
}
@@ -5226,3 +5840,53 @@ get_transaction_apply_action(TransactionId xid, ParallelApplyWorkerInfo **winfo)
return TRANS_LEADER_APPLY;
}
}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_conflict_info option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_conflict_info for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (!retain_conflict_info)
+ return;
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_conflict_info") : 0);
+}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index e44ad576bc7..63812358800 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,31 @@ 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),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ 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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 28b8591efa5..1723ab7308d 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 2418967def6..76cb0ab1de9 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2823,9 +2823,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2852,6 +2855,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2859,6 +2864,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index c6226175528..4ca52455c9e 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretainconflictinfo;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretainconflictinfo\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretainconflictinfo\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,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");
@@ -5104,6 +5113,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))
@@ -5362,6 +5373,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 39eef1d6617..02a08812a25 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,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 30579ef2051..ae1a3262ad4 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -28,7 +28,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -631,7 +631,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();
}
@@ -764,7 +764,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -2040,48 +2040,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_conflict_info enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rci_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rci_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_conflict_info option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_conflict_info)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rci_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_conflict_info
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rci_slot_on_new = PQfnumber(res, "rci_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rci_slot_on_new = atoi(PQgetvalue(res, 0, i_rci_slot_on_new));
+
+ if (rci_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_conflict_info);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -2094,12 +2126,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_conflict_info &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_conflict_info &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2211,6 +2255,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..9436fb7c310 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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretainconflictinfo THEN 1 END) > 0 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), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..69bbe41163a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_conflict_info)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_conflict_info)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_conflict_info enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..352c8b6f376 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 e46f02c6cc6..11e7e21478a 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_conflict_info
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_conflict_info = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_conflict_info
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_conflict_info
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretainconflictinfo set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretainconflictinfo FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_conflict_info are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..dbcf88be945 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ 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 37524364290..63dd4400fc6 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2319,8 +2319,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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 <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3774,8 +3775,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_proc.dat b/src/include/catalog/pg_proc.dat
index 1fc19146f46..3ee8fed7e53 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11801,6 +11801,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..0ac7c0b120c 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..d90d170054e 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,6 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/logicalworker.h b/src/include/replication/logicalworker.h
index 88912606e4d..ebbb3ef09c7 100644
--- a/src/include/replication/logicalworker.h
+++ b/src/include/replication/logicalworker.h
@@ -30,4 +30,9 @@ extern void LogicalRepWorkersWakeupAtCommit(Oid subid);
extern void AtEOXact_LogicalRepWorkers(bool isCommit);
+extern void CheckSubConflictInfoRetention(bool retain_conflict_info,
+ bool check_guc,
+ bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* LOGICALWORKER_H */
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 76aeeb92242..9020764d253 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..4fb317b3f85 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_conflict_info);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 e4877d88e8f..7cbc8bd59bd 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -55,7 +55,7 @@ extern RunningTransactions GetRunningTransactionData(void);
extern bool TransactionIdIsInProgress(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 529b2241731..2a3b672b79a 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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00012345
+ 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/00012345
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,34 @@ 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/00000000
+ 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/00000000
(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/00000000
+ 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/00000000
+(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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_conflict_info = false);
+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 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 007c9e70374..8b1b12592d0 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ 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
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..9b675b95407 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,195 @@ $node_subscriber->wait_for_log(
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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_conflict_info option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_conflict_info option.
+##################################################
+
+# Alter retain_conflict_info for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_conflict_info\" for enabled subscription/,
+ "altering retain_conflict_info is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_conflict_info for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_conflict_info = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_conflict_info is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+##################################################
+# Check the WARNING when changing the origin to ANY, if retain_conflict_info is
+# enabled. This warns of the possibility of receiving changes from origins
+# other than the publisher.
+##################################################
+
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = any);");
+ok( $stderr =~
+ /WARNING: subscription "tap_sub_a_b" enabled retain_conflict_info but might not reliably detect conflicts for changes from different origins/,
+ "warn of the possibility of receiving changes from origins other than the publisher");
+
+# Reset the origin to none
+$node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = none);");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index ff050e93a50..fe9e1edfcfb 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2565,6 +2565,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainConflictInfoData
+RetainConflictInfoPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
v49-0002-refactor-launcher-slot-creation-and-doc-perf.patchapplication/octet-stream; name=v49-0002-refactor-launcher-slot-creation-and-doc-perf.patchDownload
From 4ec216d520dd1f8fb1b66fb7124b9e0323ba5f6b Mon Sep 17 00:00:00 2001
From: Hou Zhijie <houzj.fnst@cn.fujitsu.com>
Date: Mon, 7 Jul 2025 14:28:45 +0800
Subject: [PATCH v49 2/2] refactor launcher slot creation
---
doc/src/sgml/ref/create_subscription.sgml | 37 +++++++
src/backend/replication/logical/launcher.c | 121 ++++++++++-----------
2 files changed, 95 insertions(+), 63 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 7782246727e..c1f1445dbe5 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -469,6 +469,43 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
it is recommended to disable <literal>retain_conflict_info</literal>
if the subscription will be inactive for an extended period.
</para>
+
+ <para>
+ Additionally when enabling <literal>retain_conflict_info</literal> for
+ conflict detection in logical replication, it is important to design the
+ replication topology to balance data retention requirements with
+ overall system performance. This option provides minimal performance
+ overhead when applied appropriately. The following scenarios illustrate
+ effective usage patterns when enabling this option.
+ </para>
+
+ <para>
+ a. Large Tables with Bidirectional Writes:
+ For large tables subject to concurrent writes on both publisher and
+ subscriber nodes, publishers can define row filters when creating
+ publications to segment data. This allows multiple subscriptions
+ to replicate exclusive subsets of the table, minimizing conflict
+ potential and optimizing throughput.
+ </para>
+
+ <para>
+ b. Write-Enabled Subscribers:
+ If a subscriber node is expected to perform write operations, replication
+ can be structured using multiple publications and subscriptions. By
+ distributing tables across these publications, the workload is spread among
+ several apply workers, improving concurrency and reducing contention.
+ </para>
+
+ <para>
+ c. Read-Only Subscribers:
+ In configurations involving single or multiple publisher nodes
+ performing concurrent write operations, read-only subscriber nodes may
+ replicate changes without seeing a performance impact if it does index
+ scan. However, if the subscriber is impacted due to replication lag or
+ scan performance (say due to sequential scans), it needs to follow one
+ of the two previous strategies to distribute the workload on the
+ subscriber.
+ </para>
</caution>
<para>
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f18fee0333e..e0f101784c7 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,8 +102,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
bool retain_conflict_info,
- TransactionId *xmin,
- bool *can_advance_xmin);
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
@@ -1204,22 +1203,41 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
- retain_conflict_info |= sub->retainconflictinfo;
-
- if (!sub->enabled)
+ if (sub->retainconflictinfo)
{
+ retain_conflict_info = true;
+
/*
- * This is required to ensure that we don't advance the xmin
- * of CONFLICT_DETECTION_SLOT if one of the subscriptions is
- * not enabled. Otherwise, we won't be able to detect
- * conflicts reliably for such a subscription even though it
- * has set the retain_conflict_info option.
+ * Can't advance xmin of the slot unless all the subscriptions
+ * with retain_conflict_info are enabled. This is required to
+ * ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_conflict_info option.
*/
- compute_min_nonremovable_xid(NULL, sub->retainconflictinfo,
- &xmin, &can_advance_xmin);
- continue;
+ can_advance_xmin &= sub->enabled;
+
+ /*
+ * Create a replication slot to retain information necessary
+ * for conflict detection such as dead tuples, commit
+ * timestamps, and origins.
+ *
+ * The slot is created before starting the apply worker to
+ * prevent it from unnecessarily maintaining its
+ * oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to
+ * ensure that conflict-related information is available when
+ * applying remote changes that occurred before the
+ * subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
}
+ if (!sub->enabled)
+ continue;
+
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
@@ -1228,12 +1246,20 @@ ApplyLauncherMain(Datum main_arg)
* Compute the minimum xmin required to protect deleted tuples
* required for conflict detection.
*/
- compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin,
- &can_advance_xmin);
+ if (can_advance_xmin)
+ compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin);
if (w != NULL)
continue; /* worker is running already */
+ /*
+ * Can't advance xmin of the slot unless all the workers
+ * corresponding to subscriptions with retain_conflict_info are
+ * running.
+ */
+ if (sub->retainconflictinfo)
+ can_advance_xmin = false;
+
/*
* If the worker is eligible to start now, launch it. Otherwise,
* adjust wait_time so that we'll wake up as soon as it can be
@@ -1322,62 +1348,31 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_conflict_info enabled. Store the result
* in *xmin.
- *
- * If the replication slot cannot be advanced during this cycle, due to either
- * a disabled subscription or an inactive worker, set *can_advance_xmin to
- * false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info, TransactionId *xmin,
- bool *can_advance_xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool retain_conflict_info,
+ TransactionId *xmin)
{
- if (!retain_conflict_info || !*can_advance_xmin)
- return;
+ TransactionId nonremovable_xid;
- if (worker)
- {
- TransactionId nonremovable_xid;
+ if (!retain_conflict_info || !worker)
+ return;
- /*
- * The replication slot for conflict detection must be created before
- * the worker starts.
- */
- Assert(MyReplicationSlot);
+ /*
+ * The replication slot for conflict detection must be created before the
+ * worker starts.
+ */
+ Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
- nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ Assert(TransactionIdIsValid(nonremovable_xid));
- if (!TransactionIdIsValid(*xmin) ||
- TransactionIdPrecedes(nonremovable_xid, *xmin))
- *xmin = nonremovable_xid;
- }
- else
- {
- /*
- * Create a replication slot to retain information necessary for
- * conflict detection such as dead tuples, commit timestamps, and
- * origins.
- *
- * The slot is created before starting the apply worker to prevent it
- * from unnecessarily maintaining its oldest_nonremovable_xid.
- *
- * The slot is created even for a disabled subscription to ensure that
- * conflict-related information is available when applying remote
- * changes that occurred before the subscription was enabled.
- */
- CreateConflictDetectionSlot();
-
- /*
- * Can't advance xmin of the slot unless all the subscriptions with
- * retain_conflict_info are enabled and the corresponding workers are
- * running.
- */
- *can_advance_xmin = false;
- }
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
}
/*
--
2.50.1.windows.1
On Mon, Jul 7, 2025 at 3:31 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Mon, Jul 7, 2025 at 10:13 AM Zhijie Hou (Fujitsu) wrote:
On Sun, Jul 6, 2025 at 10:51 PM Masahiko Sawada wrote:
On Fri, Jul 4, 2025 at 8:18 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:On Wed, Jul 2, 2025 at 3:28 PM Hou, Zhijie wrote:
Kindly use the latest patch set for performance testing.
During testing, we observed a limitation in cascading logical
replication setups, such as (A -> B -> C). When retain_conflict_info
is enabled on Node C, it may not retain information necessary for
conflict detection when applying changes originally replicated from
Node A. This happens because Node C only waits for locally
originated changes on Node B to be applied before advancing the
non-removabletransaction ID.
For example, Consider a logical replication setup as mentioned above
: A -> B-> C.
- All three nodes have a table t1 with two tuples (1,1) (2,2).
- Node B subscribed to all changes of t1 from Node A
- Node-C subscribed to all changes from Node B.
- Subscriptions use the default origin=ANY, as this is not a bidirectional
setup.Now, consider two concurrent operations:
- @9:00 Node A - UPDATE (1,1) -> (1,11)- @9:02 Node C - DELETE (1,1)
Assume a slight delay at Node B before it applies the update from Node A.
@9:03 Node C - advances the non-removable XID because it sees no
concurrent transactions from Node B. It is unaware of Node A’s
concurrentupdate.
@9:04 Node B - receives Node A's UPDATE and applies (1,1) -> (1,11)
t1 has tuples : (1,11), (2,2)@9:05 Node C - receives the UPDATE (1,1) -> (1,11)
- As conflict slot’s xmin is advanced, the deleted tuple may
alreadyhave
been removed.
- Conflict resolution fails to detect update_deleted and instead raises
update_missing.Note that, as per decoding logic Node C sees the commit timestamp of
the update as 9:00 (origin commit_ts from Node A), not 9:04 (commit
time on Node B). In this case, since the UPDATE's timestamp is
earlier than the DELETE, Node C should ideally detect an
update_deleted conflict. However, it cannot, because it no longer retainsthe deleted tuple.
Even if Node C attempts to retrieve the latest WAL position from
Node A, Node C doesn't maintain any LSN which we could use to comparewith it.
This scenario is similar to another restriction in the patch where
retain_conflict_info is not supported if the publisher is also a
physical standby, as the required transaction information from the
original primary is unavailable. Moreover, this limitation is
relevant only when the subscription origin option is set to ANY, as
only in that case changes from other origins can be replicated.
Since retain_conflict_info is primarily useful for conflict
detection in bidirectional clusters where the origin option is set
to NONE, this limitationappears acceptable.
Given these findings, to help users avoid unintended configurations,
we plan to issue a warning in scenarios where replicated changes may
include origins other than the direct publisher, similar to the
existing checks in the
check_publications_origin() function.Here is the latest patch that implements the warning and documents
this case. Only 0001 is modified for this.A big thanks to Nisha for invaluable assistance in identifying this
case and preparing the analysis for it.I'm still reviewing the 0001 patch but let me share some comments and
questions I have so far:Thanks for the comments!
--- +/* + * Determine the minimum non-removable transaction ID across all +apply workers + * for subscriptions that have retain_conflict_info enabled. Store +the result + * in *xmin. + * + * If the replication slot cannot be advanced during this cycle, due +to either + * a disabled subscription or an inactive worker, set +*can_advance_xmin to + * false. + */ +static void +compute_min_nonremovable_xid(LogicalRepWorker *worker, + bool retain_conflict_info, TransactionId*xmin,
+ bool *can_advance_xmin)
I think this function is quite confusing for several reasons. For
instance, it's doing more things than described in the comments such
as trying to create the CONFLICT_DETECTION_SLOT if no worker is
passed. Also, one of the caller
describes:+ /* + * This is required to ensure that we don't advance the xmin + * of CONFLICT_DETECTION_SLOT even if one of the subscriptions + * is not enabled. Otherwise, we won't be able to detect + * conflicts reliably for such a subscription even though it + * has set the retain_conflict_info option. + */ + compute_min_nonremovable_xid(NULL, sub->retainconflictinfo, + &xmin, + &can_advance_xmin);but it's unclear to me from the function name that it tries to create
the replication slot. Furthermore, in this path it doesn't actually
compute xmin. I guess we can try to create CONFLICT_DETECTION_SLOT in
the loop of "foreach(lc, sublist)" and set false to can_advance_xmin
if either the subscription is disabled or the worker is not running.I understand. The original code was similar to your suggestion, but we decided
to encapsulate it within a separate function to maintain a clean and concise
main loop. However, your suggestion also makes sense, so I will proceed with
the change.I have made this change in the 0002 patch for reference. What do you think ? If
there are no objections, I plan to merge it in the next version.
The changes in the 0002 patch look good to me. I've attached the patch
for some minor suggestions. Please incorporate these changes if you
agree.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
Attachments:
change_masahiko_v49.patchapplication/octet-stream; name=change_masahiko_v49.patchDownload
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index e0f101784c7..005098932dc 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -100,9 +100,7 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker,
- bool retain_conflict_info,
- TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
@@ -1193,7 +1191,14 @@ ApplyLauncherMain(Datum main_arg)
ALLOCSET_DEFAULT_SIZES);
oldctx = MemoryContextSwitchTo(subctx);
- /* Start any missing workers for enabled subscriptions. */
+ /*
+ * Start any missing workers for enabled subscriptions.
+ *
+ * Also, during the iteration through all subscriptions, we compute
+ * the minimum XID required to protect deleted tuples for conflict
+ * detection if one of the subscription enables retain_conflict_info
+ * option.
+ */
sublist = get_subscription_list();
foreach(lc, sublist)
{
@@ -1242,20 +1247,25 @@ ApplyLauncherMain(Datum main_arg)
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
LWLockRelease(LogicalRepWorkerLock);
- /*
- * Compute the minimum xmin required to protect deleted tuples
- * required for conflict detection.
- */
- if (can_advance_xmin)
- compute_min_nonremovable_xid(w, sub->retainconflictinfo, &xmin);
-
if (w != NULL)
- continue; /* worker is running already */
+ {
+ /*
+ * Compute the minimum xmin required to protect deleted tuples
+ * required for conflict detection among all running apply
+ * workers that enables retain_conflict_info.
+ */
+ if (sub->retainconflictinfo && can_advance_xmin)
+ compute_min_nonremovable_xid(w, &xmin);
+
+ /* worker is running already */
+ continue;
+ }
/*
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_conflict_info are
- * running.
+ * running, disabling the further computation of the minimum
+ * nonremovable xid.
*/
if (sub->retainconflictinfo)
can_advance_xmin = false;
@@ -1350,13 +1360,11 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, bool retain_conflict_info,
- TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
{
TransactionId nonremovable_xid;
- if (!retain_conflict_info || !worker)
- return;
+ Assert(worker != NULL);
/*
* The replication slot for conflict detection must be created before the
On Fri, Jul 18, 2025 at 5:03 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, July 18, 2025 1:25 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jul 11, 2025 at 3:58 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jul 10, 2025 at 6:46 PM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:
On Wed, Jul 9, 2025 at 9:09 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
I think that even with retain_conflict_info = off, there is
probably a point at which the subscriber can no longer keep up
with the publisher. For example, if with retain_conflict_info =
off we can withstand 100 clients running at the same time, then
the fact that this performance degradation occurred with 15
clients explains that performance degradation is much more
likely to occur because of retain_conflict_info = on.Test cases 3 and 4 are typical cases where this feature is used
since the conflicts actually happen on the subscriber, so I
think it's important to look at the performance in these cases.
The worst case scenario for this feature is that when this
feature is turned on, the subscriber cannot keep up even with a
small load, and with max_conflict_retetion_duration we enter a
loop of slot invalidation and re-creating, which means that
conflict cannot be detected reliably.As per the above observations, it is less of a regression of this
feature but more of a lack of parallel apply or some kind of
pre-fetch for apply, as is recently proposed [1]. I feel there are
use cases, as explained above, for which this feature would work
without any downside, but due to a lack of some sort of parallel
apply, we may not be able to use it without any downside for cases
where the contention is only on a smaller set of tables. We have
not tried, but may in cases where contention is on a smaller set
of tables, if users distribute workload among different pub-sub
pairs by using row filters, there also, we may also see less
regression. We can try that as well.While I understand that there are some possible solutions we have
today to reduce the contention, I'm not really sure these are really
practical solutions as it increases the operational costs instead.I assume by operational costs you mean defining the replication
definitions such that workload is distributed among multiple apply
workers via subscriptions either by row_filters, or by defining
separate pub-sub pairs of a set of tables, right? If so, I agree with
you but I can't think of a better alternative. Even without this
feature as well, we know in such cases the replication lag could be
large as is evident in recent thread [1] and some offlist feedback by
people using native logical replication. As per a POC in the
thread[1], parallelizing apply or by using some prefetch, we could
reduce the lag but we need to wait for that work to mature to see the
actual effect of it.I don't have a better alternative either.
I agree that this feature will work without any problem when logical replication
is properly configured. It's a good point that update-delete conflicts can be
detected reliably without additional performance overhead in scenarios with
minimal replication lag.
However, this approach requires users to carefully pay particular attention to
replication performance and potential delays. My primary concern is that, given
the current logical replication performance limitations, most users who want to
use this feature will likely need such dedicated care for replication lag.
Nevertheless, most features involve certain trade-offs. Given that this is an
opt-in feature and future performance improvements will reduce these
challenges for users, it would be reasonable to have this feature at this stage.The path I see with this work is to clearly document the cases
(configuration) where this feature could be used without much downside
and keep the default value of subscription option to enable this as
false (which is already the case with the patch).+1
Thanks for the discussion. Here is the V49 patch which includes the suggested
doc change in 0002. I will rebase the remaining patches once the first one is
pushed.
Thank you for updating the patch!
Here are some review comments and questions:
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsReservedSlotName(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection
and can only be acquired by logical replication launcher."));
I think it might be better to rename IsReservedSlotName() to be more
specific to the conflict detection because we might want to add more
reserved slot names in the future that would not necessarily be
acquired only by the launcher process.
---
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
I've not verified yet but is it possible that we exclude XIDs of
processes who are running on other databases?
---
Regarding the option name we've discussed:
The new parameter name "retain_conflict_info" sounds to me like we keep the
conflict information somewhere that has expired at some time such as how
many times insert_exists or update_origin_differs happened. How about
choosing a name that indicates retain dead tuples more explicitly for example
retain_dead_tuples?We considered the name you suggested, but we wanted to convey that this option
not only retains dead tuples but also preserves commit timestamps and origin
data for conflict detection, hence we opted for a more general name. Do you
have better suggestions?
I think the commit timestamp and origin data would be retained as a
result of retaining dead tuples. While such a general name could
convey more than retaining dead tuples, I'm concerned that it could be
ambiguous what exactly to retain by the subscription. How about the
following names or something along those lines?
- retain_dead_tuples_for_conflict
- delay_vacuum_for_conflict
- keep_dead_tuples_for_conflict
---
+check_pub_conflict_info_retention(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;
It seems that retain_conflict_info is used only for this check to
quick exit from this function. How about calling this function only
when the caller knows retain_conflict_info is true instead of adding
it as a function argument?
---
+void
+CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc,
+ bool sub_disabled, int elevel_for_sub_disabled)
This function seems not to be specific to the apply workers but to
subscriptions in general. Is there any reason why we define this
function in worker.c?
---
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the
replication slot required by retain_conflict_info"),
+ errhint("\"wal_level\" must be set to \"replica\" or
\"logical\" at server start."));
+
Why does (retain_conflict_info == false && wal_level == minimal) not work?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Sat, Jul 19, 2025 at 3:01 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jul 18, 2025 at 5:03 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here are some review comments and questions:
--- + if (inCommitOnly && + (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0) + continue; +I've not verified yet but is it possible that we exclude XIDs of
processes who are running on other databases?
I can't see how, even the comments atop function says: " We look at
all databases, though there is no need to include WALSender since this
has no effect on hot standby conflicts." which indicate that it
shouldn't exlude XIDs of procs who are running on other databases.
---
Regarding the option name we've discussed:The new parameter name "retain_conflict_info" sounds to me like we keep the
conflict information somewhere that has expired at some time such as how
many times insert_exists or update_origin_differs happened. How about
choosing a name that indicates retain dead tuples more explicitly for example
retain_dead_tuples?We considered the name you suggested, but we wanted to convey that this option
not only retains dead tuples but also preserves commit timestamps and origin
data for conflict detection, hence we opted for a more general name. Do you
have better suggestions?I think the commit timestamp and origin data would be retained as a
result of retaining dead tuples. While such a general name could
convey more than retaining dead tuples, I'm concerned that it could be
ambiguous what exactly to retain by the subscription. How about the
following names or something along those lines?- retain_dead_tuples_for_conflict
- delay_vacuum_for_conflict
- keep_dead_tuples_for_conflict
Among these, the first option is better but I think it is better to
name it just retain_dead_tuples. The explanation of the option will
explain its use. It is similar to other options like binary or
streaming. We are not naming them like request_data_binary_format to
make the meaning apparent. There is a value in keeping names succinct.
--
With Regards,
Amit Kapila.
On Saturday, July 19, 2025 5:31 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Here are some review comments and questions:
Thanks for the comments!
+ /* + * Do not allow users to acquire the reserved slot. This scenario may + * occur if the launcher that owns the slot has terminated unexpectedly + * due to an error, and a backend process attempts to reuse the slot. + */ + if (!IsLogicalLauncher() && IsReservedSlotName(name)) + ereport(ERROR, + errcode(ERRCODE_UNDEFINED_OBJECT), + errmsg("cannot acquire replication slot \"%s\"", name), + errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));I think it might be better to rename IsReservedSlotName() to be more specific to
the conflict detection because we might want to add more reserved slot names
in the future that would not necessarily be acquired only by the launcher
process.
Agreed. I have renamed it to IsSlotForConflictCheck.
---
Regarding the option name we've discussed:The new parameter name "retain_conflict_info" sounds to me like we
keep the conflict information somewhere that has expired at some
time such as how many times insert_exists or update_origin_differs
happened. How about choosing a name that indicates retain dead
tuples more explicitly for example retain_dead_tuples?We considered the name you suggested, but we wanted to convey that
this option not only retains dead tuples but also preserves commit
timestamps and origin data for conflict detection, hence we opted for
a more general name. Do you have better suggestions?I think the commit timestamp and origin data would be retained as a result of
retaining dead tuples. While such a general name could convey more than
retaining dead tuples, I'm concerned that it could be ambiguous what exactly to
retain by the subscription. How about the following names or something along
those lines?- retain_dead_tuples_for_conflict
- delay_vacuum_for_conflict
- keep_dead_tuples_for_conflict
OK, I use the shorter version retain_conflict_info as mentioned by Amit[1]/messages/by-id/CAA4eK1JdCJK0KtV5WGYWoQVe7S3uqx-7J7t1qFpCki_rdLQFmw@mail.gmail.com.
--- +check_pub_conflict_info_retention(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;It seems that retain_conflict_info is used only for this check to quick exit from
this function. How about calling this function only when the caller knows
retain_conflict_info is true instead of adding it as a function argument?
Changed as suggested.
--- +void +CheckSubConflictInfoRetention(bool retain_conflict_info, bool check_guc, + bool sub_disabled, int +elevel_for_sub_disabled)This function seems not to be specific to the apply workers but to
subscriptions in general. Is there any reason why we define this function in
worker.c?
I do not have special reasons, so I moved it to subscriptioncmd.c
where most checks are performed.
--- + if (check_guc && wal_level < WAL_LEVEL_REPLICA) + ereport(ERROR, + errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE), + errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_conflict_info"), + errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start.")); +Why does (retain_conflict_info == false && wal_level == minimal) not work?
I think it works because the check is skipped when rci is false. BTW, to be consistent with
check_pub_conflict_info_retention, I moved the retain_conflict_info check outside of this
function.
[1]: /messages/by-id/CAA4eK1JdCJK0KtV5WGYWoQVe7S3uqx-7J7t1qFpCki_rdLQFmw@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v50-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v50-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From 86e27d2dd94b60c6fe67b7fb29d3cc0bba2937aa Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 18 Jul 2025 17:31:20 +0800
Subject: [PATCH v50] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_dead_tuples option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_dead_tuples enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 87 ++-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 400 ++++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 228 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 617 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 56 ++
src/backend/storage/ipc/procarray.c | 11 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 5 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 2 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 195 +++++-
src/tools/pgindent/typedefs.list | 2 +
44 files changed, 2214 insertions(+), 217 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 0d23bc1b122..97f547b3cc4 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretaindeadtuples</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index c7acc0f182f..20ccb2d6b54 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4965,6 +4965,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index f5a0e0954a1..de5b5929ee0 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29592,7 +29592,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29636,7 +29638,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29666,6 +29670,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29688,8 +29694,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index e26f7f59d4a..65b090cb3dd 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2396,6 +2396,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2532,6 +2538,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2563,6 +2585,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_dead_tuples enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index e74b5be1eff..b115884acb3 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2235,6 +2235,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2653,6 +2655,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2797,6 +2858,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..d48cdc76bd3 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ dead tuples for conflict detection, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..b8cd15f3280 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,89 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-dead-tuples">
+ <term><literal>retain_dead_tuples</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_dead_tuples</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ Additionally when enabling <literal>retain_dead_tuples</literal> for
+ conflict detection in logical replication, it is important to design the
+ replication topology to balance data retention requirements with
+ overall system performance. This option provides minimal performance
+ overhead when applied appropriately. The following scenarios illustrate
+ effective usage patterns when enabling this option.
+ </para>
+
+ <para>
+ a. Large Tables with Bidirectional Writes:
+ For large tables subject to concurrent writes on both publisher and
+ subscriber nodes, publishers can define row filters when creating
+ publications to segment data. This allows multiple subscriptions
+ to replicate exclusive subsets of the table in parallel, optimizing
+ the throughput.
+ </para>
+
+ <para>
+ b. Write-Enabled Subscribers:
+ If a subscriber node is expected to perform write operations, replication
+ can be structured using multiple publications and subscriptions. By
+ distributing tables across these publications, the workload is spread among
+ several apply workers, improving concurrency and reducing contention.
+ </para>
+
+ <para>
+ c. Read-Only Subscribers:
+ In configurations involving single or multiple publisher nodes
+ performing concurrent write operations, read-only subscriber nodes may
+ replicate changes without seeing a performance impact if it does index
+ scan. However, if the subscriber is impacted due to replication lag or
+ scan performance (say due to sequential scans), it needs to follow one
+ of the two previous strategies to distribute the workload on the
+ subscriber.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+
+ <para>
+ Enabling this option ensures retention of information useful for
+ conflict detection solely for changes occurring locally on the
+ publisher. For the changes originating from different origins,
+ reliable conflict detection cannot be guaranteed.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 85cbe397cb2..7c94d020d7a 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1183,7 +1183,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 in logical replication. See
+ * GetOldestActiveTransactionId(true) and its use.
*
* We save the PREPARE record's location in the gxact for later use by
* CheckPointTwoPhase.
@@ -2298,7 +2302,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2318,7 +2322,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2331,8 +2335,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 comments atop worker.c.
+ */
+ committs = GetCurrentTimestamp();
/*
* Emit the XLOG commit record. Note that we mark 2PC commits as
@@ -2381,7 +2401,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 41601fcb280..b46e7e9c2a6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 8e7827c6ed9..ed7d194ab7f 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7121,7 +7121,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 23878b2dd91..e8f3ba00caa 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..63c2992d19f 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->retaindeadtuples = subform->subretaindeadtuples;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index b2d5332effc..f6eca09ee15 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1386,7 +1386,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)
+ subretaindeadtuples, 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 e23b0de7242..cd6c3684482 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_DEAD_TUPLES 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 retaindeadtuples;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -105,8 +108,10 @@ typedef struct SubOpts
static List *fetch_table_list(WalReceiverConn *wrconn, List *publications);
static void check_publications_origin(WalReceiverConn *wrconn,
List *publications, bool copydata,
- char *origin, Oid *subrel_local_oids,
- int subrel_count, char *subname);
+ bool retain_dead_tuples, char *origin,
+ Oid *subrel_local_oids, int subrel_count,
+ char *subname);
+static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
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_DEAD_TUPLES))
+ opts->retaindeadtuples = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -210,7 +217,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)
@@ -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_DEAD_TUPLES) &&
+ strcmp(defel->defname, "retain_dead_tuples") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
+ opts->retaindeadtuples = 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_DEAD_TUPLES | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retain_dead_tuples */
+ if (opts.retaindeadtuples)
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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_subretaindeadtuples - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -722,7 +745,11 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
{
check_publications(wrconn, publications);
check_publications_origin(wrconn, publications, opts.copy_data,
- opts.origin, NULL, 0, stmt->subname);
+ opts.retaindeadtuples, opts.origin,
+ NULL, 0, stmt->subname);
+
+ if (opts.retaindeadtuples)
+ check_pub_dead_tuple_retention(wrconn);
/*
* Set sync state based on if we were asked to do data copy or
@@ -881,8 +908,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
sizeof(Oid), oid_cmp);
check_publications_origin(wrconn, sub->publications, copy_data,
- sub->origin, subrel_local_oids,
- subrel_count, sub->name);
+ sub->retaindeadtuples, sub->origin,
+ subrel_local_oids, subrel_count, sub->name);
/*
* Rels that we want to remove from subscription and drop any slots
@@ -1040,18 +1067,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_dead_tuples
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_dead_tuples") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_dead_tuples option does not update the slot on the
+ * publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_dead_tuples") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1094,39 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_dead_tuples option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_dead_tuples option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_dead_tuples concurrently with the
+ * launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * retreat in the calculated xmin, necessitating additional handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the retain_dead_tuples,
+ * but in the future we can change this after some more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_dead_tuples to
+ * avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_tuple = false;
bool update_failover = false;
bool update_two_phase = false;
+ bool check_pub_rdt = false;
+ bool retain_dead_tuples;
+ char *origin;
Subscription *sub;
Form_pg_subscription form;
bits32 supported_opts;
@@ -1137,6 +1204,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
sub = GetSubscription(subid, false);
+ retain_dead_tuples = sub->retaindeadtuples;
+ origin = sub->origin;
+
/*
* Don't allow non-superuser modification of a subscription with
* password_required=false.
@@ -1165,7 +1235,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,11 +1395,62 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
+ {
+ values[Anum_pg_subscription_subretaindeadtuples - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+
+ CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ if (opts.retaindeadtuples)
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
+
+ /*
+ * 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();
+
+ check_pub_rdt = opts.retaindeadtuples;
+ retain_dead_tuples = opts.retaindeadtuples;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
CStringGetTextDatum(opts.origin);
replaces[Anum_pg_subscription_suborigin - 1] = true;
+
+ /*
+ * Check if changes from different origins may be received
+ * from the publisher when the origin is changed to ANY
+ * and retain_dead_tuples is enabled.
+ */
+ check_pub_rdt = retain_dead_tuples &&
+ pg_strcasecmp(opts.origin, LOGICALREP_ORIGIN_ANY) == 0;
+
+ origin = opts.origin;
}
update_tuple = true;
@@ -1347,6 +1468,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubDeadTupleRetention() for details.
+ */
+ if (sub->retaindeadtuples)
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1485,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
ApplyLauncherWakeupAtCommit();
update_tuple = true;
+
+ /*
+ * The subscription might be initially created with
+ * connect=false and retain_dead_tuples=true, meaning the
+ * remote server's status may not be checked. Ensure this
+ * check is conducted now.
+ */
+ check_pub_rdt = sub->retaindeadtuples && opts.enabled;
break;
}
@@ -1369,6 +1507,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_dead_tuples.
+ */
+ check_pub_rdt = sub->retaindeadtuples;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1713,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_dead_tuples.
*
* 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 || check_pub_rdt)
{
bool must_use_password;
char *err;
@@ -1584,10 +1730,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 +1746,17 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
PG_TRY();
{
- walrcv_alter_slot(wrconn, sub->slotname,
- update_failover ? &opts.failover : NULL,
- update_two_phase ? &opts.twophase : NULL);
+ if (retain_dead_tuples)
+ check_pub_dead_tuple_retention(wrconn);
+
+ check_publications_origin(wrconn, sub->publications, false,
+ retain_dead_tuples, origin, NULL, 0,
+ sub->name);
+
+ 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();
{
@@ -2086,20 +2244,29 @@ AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId)
* Check and log a warning if the publisher has subscribed to the same table,
* its partition ancestors (if it's a partition), or its partition children (if
* it's a partitioned table), from some other publishers. This check is
- * required only if "copy_data = true" and "origin = none" for CREATE
- * SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements to notify the
- * user that data having origin might have been copied.
+ * required in the following scenarios:
*
- * This check need not be performed on the tables that are already added
- * because incremental sync for those tables will happen through WAL and the
- * origin of the data can be identified from the WAL records.
+ * 1) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "copy_data = true" and "origin = none":
+ * - Warn the user that data with an origin might have been copied.
+ * - This check is skipped for tables already added, as incremental sync via
+ * WAL allows origin tracking. The list of such tables is in
+ * subrel_local_oids.
*
- * subrel_local_oids contains the list of relation oids that are already
- * present on the subscriber.
+ * 2) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "retain_dead_tuples = true" and "origin = any", and for ALTER
+ * SUBSCRIPTION statements that modify retain_dead_tuples or origin, or
+ * when the publisher's status changes (e.g., due to a connection string
+ * update):
+ * - Warn the user that only conflict detection info for local changes on
+ * the publisher is retained. Data from other origins may lack sufficient
+ * details for reliable conflict detection.
+ * - See comments atop worker.c for more details.
*/
static void
check_publications_origin(WalReceiverConn *wrconn, List *publications,
- bool copydata, char *origin, Oid *subrel_local_oids,
+ bool copydata, bool retain_dead_tuples,
+ char *origin, Oid *subrel_local_oids,
int subrel_count, char *subname)
{
WalRcvExecResult *res;
@@ -2108,9 +2275,29 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
Oid tableRow[1] = {TEXTOID};
List *publist = NIL;
int i;
+ bool check_rdt;
+ bool check_table_sync;
+ bool origin_none = origin &&
+ pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) == 0;
+
+ /*
+ * Enable retain_dead_tuples checks only when origin is set to 'any',
+ * since with origin='none' only local changes are replicated to the
+ * subscriber.
+ */
+ check_rdt = retain_dead_tuples && !origin_none;
+
+ /*
+ * Enable table synchronization checks only when origin is 'none', to
+ * ensure that data from other origins is not inadvertently copied.
+ */
+ check_table_sync = copydata && origin_none;
- if (!copydata || !origin ||
- (pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) != 0))
+ /* retain_dead_tuples and table sync checks occur separately */
+ Assert(!(check_rdt && check_table_sync));
+
+ /* Return if no checks are required */
+ if (!check_rdt && !check_table_sync)
return;
initStringInfo(&cmd);
@@ -2129,16 +2316,23 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
/*
* In case of ALTER SUBSCRIPTION ... REFRESH, subrel_local_oids contains
* the list of relation oids that are already present on the subscriber.
- * This check should be skipped for these tables.
+ * This check should be skipped for these tables if checking for table
+ * sync scenario. However, when handling the retain_dead_tuples scenario,
+ * ensure all tables are checked, as some existing tables may now include
+ * changes from other origins due to newly created subscriptions on the
+ * publisher.
*/
- for (i = 0; i < subrel_count; i++)
+ if (check_table_sync)
{
- Oid relid = subrel_local_oids[i];
- char *schemaname = get_namespace_name(get_rel_namespace(relid));
- char *tablename = get_rel_name(relid);
+ for (i = 0; i < subrel_count; i++)
+ {
+ Oid relid = subrel_local_oids[i];
+ char *schemaname = get_namespace_name(get_rel_namespace(relid));
+ char *tablename = get_rel_name(relid);
- appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
- schemaname, tablename);
+ appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
+ schemaname, tablename);
+ }
}
res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
@@ -2173,22 +2367,37 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
* XXX: For simplicity, we don't check whether the table has any data or
* not. If the table doesn't have any data then we don't need to
* distinguish between data having origin and data not having origin so we
- * can avoid logging a warning in that case.
+ * can avoid logging a warning for table sync scenario.
*/
if (publist)
{
StringInfo pubnames = makeStringInfo();
+ StringInfo err_msg = makeStringInfo();
+ StringInfo err_hint = makeStringInfo();
/* Prepare the list of publication(s) for warning message. */
GetPublicationsStr(publist, pubnames, false);
+
+ if (check_table_sync)
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin"),
+ subname);
+ appendStringInfoString(err_hint, _("Verify that initial data copied from the publisher tables did not come from other origins."));
+ }
+ else
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" enabled retain_dead_tuples but might not reliably detect conflicts for changes from different origins"),
+ subname);
+ appendStringInfoString(err_hint, _("Consider using origin = NONE or disabling retain_dead_tuples."));
+ }
+
ereport(WARNING,
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin",
- subname),
- errdetail_plural("The subscription being created subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
- "The subscription being created subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
+ errmsg_internal("%s", err_msg->data),
+ errdetail_plural("The subscription subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
+ "The subscription subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
list_length(publist), pubnames->data),
- errhint("Verify that initial data copied from the publisher tables did not come from other origins."));
+ errhint_internal("%s", err_hint->data));
}
ExecDropSingleTupleTableSlot(slot);
@@ -2196,6 +2405,101 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_dead_tuples can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop worker.c for a detailed explanation.
+ */
+static void
+check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (walrcv_server_version(wrconn) < 19000)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot enable retain_dead_tuples if the publisher is running a version earlier than PostgreSQL 19"));
+
+ 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_dead_tuples if the publisher is in recovery."));
+
+ ExecDropSingleTupleTableSlot(slot);
+
+ walrcv_clear_result(res);
+}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_dead_tuples option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_dead_tuples for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
+ int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..1f8948e2f27 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +151,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->retaindeadtuples = subform->subretaindeadtuples;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -309,7 +313,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_dead_tuples)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +333,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_dead_tuples is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_dead_tuples || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +462,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_dead_tuples
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1129,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1164,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1179,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1168,7 +1191,14 @@ ApplyLauncherMain(Datum main_arg)
ALLOCSET_DEFAULT_SIZES);
oldctx = MemoryContextSwitchTo(subctx);
- /* Start any missing workers for enabled subscriptions. */
+ /*
+ * Start any missing workers for enabled subscriptions.
+ *
+ * Also, during the iteration through all subscriptions, we compute
+ * the minimum XID required to protect deleted tuples for conflict
+ * detection if one of the subscription enables retain_dead_tuples
+ * option.
+ */
sublist = get_subscription_list();
foreach(lc, sublist)
{
@@ -1178,6 +1208,38 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ if (sub->retaindeadtuples)
+ {
+ retain_dead_tuples = true;
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions
+ * with retain_dead_tuples are enabled. This is required to
+ * ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ can_advance_xmin &= sub->enabled;
+
+ /*
+ * Create a replication slot to retain information necessary
+ * for conflict detection such as dead tuples, commit
+ * timestamps, and origins.
+ *
+ * The slot is created before starting the apply worker to
+ * prevent it from unnecessarily maintaining its
+ * oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to
+ * ensure that conflict-related information is available when
+ * applying remote changes that occurred before the
+ * subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+ }
+
if (!sub->enabled)
continue;
@@ -1186,7 +1248,27 @@ ApplyLauncherMain(Datum main_arg)
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
- continue; /* worker is running already */
+ {
+ /*
+ * Compute the minimum xmin required to protect dead tuples
+ * required for conflict detection among all running apply
+ * workers that enables retain_dead_tuples.
+ */
+ if (sub->retaindeadtuples && can_advance_xmin)
+ compute_min_nonremovable_xid(w, &xmin);
+
+ /* worker is running already */
+ continue;
+ }
+
+ /*
+ * Can't advance xmin of the slot unless all the workers
+ * corresponding to subscriptions with retain_dead_tuples are
+ * running, disabling the further computation of the minimum
+ * nonremovable xid.
+ */
+ if (sub->retaindeadtuples)
+ can_advance_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1210,7 +1292,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retaindeadtuples))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1313,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain dead tuples. Otherwise, if required,
+ * advance the slot's xmin to protect dead tuples required for the
+ * conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_dead_tuples)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1354,125 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_dead_tuples enabled. Store the result
+ * in *xmin.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+{
+ TransactionId nonremovable_xid;
+
+ Assert(worker != NULL);
+
+ /*
+ * The replication slot for conflict detection must be created before the
+ * worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7b4e8629553..5febd154b6b 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index e4fd6347fd1..3fea0a0206e 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c5fb627aa56..41be301deb4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -132,6 +132,94 @@
* failover = true when creating the subscription. Enabling failover allows us
* to smoothly transition to the promoted standby, ensuring that we can
* subscribe to the new primary without losing any data.
+ *
+ * RETAIN DEAD TUPLES
+ * ----------------------
+ * Each apply worker that enabled retain_dead_tuples option maintains a
+ * non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
+ * prevent dead rows from being removed prematurely when the apply worker still
+ * needs them to detect conflicts reliably. This helps to retain the required
+ * commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * The logical replication launcher manages an internal replication slot named
+ * "pg_conflict_detection". It asynchronously aggregates the non-removable
+ * transaction ID from all apply workers to determine the appropriate xmin for
+ * the slot, thereby retaining necessary tuples.
+ *
+ * The non-removable transaction ID in the apply worker is advanced to the
+ * oldest running transaction ID once all concurrent transactions on the
+ * publisher have been applied and flushed locally. The process involves:
+ *
+ * - RDT_GET_CANDIDATE_XID:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid.
+ *
+ * - RDT_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.
+ *
+ * - RDT_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 (RDT_REQUEST_PUBLISHER_STATUS)
+ * until all these transactions have completed.
+ *
+ * - RDT_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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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.
+ *
+ * Similarly, when the publisher has subscribed to another publisher,
+ * information necessary for conflict detection cannot be retained for
+ * changes from origins other than the publisher. This is because the publisher
+ * lacks the transaction status of other publishers it subscribes to.
+ * Consequently, the non-removable transaction ID might be advanced prematurely
+ * before changes from other origins have been fully applied.
+ *
+ * XXX Retaining information for changes from other origins might be possible
+ * by requesting the subscription on that origin to enable retain_dead_tuples
+ * and fetching the conflict detection slot.xmin along with the publisher's
+ * status. In the RDT_WAIT_FOR_PUBLISHER_STATUS phase, the apply worker could
+ * wait for the remote slot's xmin to reach the oldest active transaction ID,
+ * ensuring that all transactions from other origins have been applied on the
+ * publisher, thereby getting the latest WAL position that includes all
+ * concurrent changes. However, this approach may impact performance, so it
+ * might not worth the effort.
+ *
+ * XXX It seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
*-------------------------------------------------------------------------
*/
@@ -140,6 +228,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -148,6 +237,7 @@
#include "catalog/pg_inherits.h"
#include "catalog/pg_subscription.h"
#include "catalog/pg_subscription_rel.h"
+#include "commands/subscriptioncmds.h"
#include "commands/tablecmds.h"
#include "commands/trigger.h"
#include "executor/executor.h"
@@ -166,12 +256,14 @@
#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"
#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"
@@ -268,6 +360,78 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * See comments atop worker.c for details of the transition between these
+ * phases.
+ */
+typedef enum
+{
+ RDT_GET_CANDIDATE_XID,
+ RDT_REQUEST_PUBLISHER_STATUS,
+ RDT_WAIT_FOR_PUBLISHER_STATUS,
+ RDT_WAIT_FOR_LOCAL_FLUSH
+} RetainDeadTuplesPhase;
+
+/*
+ * Critical information for managing phase transitions within the
+ * RetainDeadTuplesPhase.
+ */
+typedef struct RetainDeadTuplesData
+{
+ RetainDeadTuplesPhase phase; /* current phase */
+ XLogRecPtr remote_lsn; /* WAL write position on the publisher */
+
+ /*
+ * Oldest transaction ID that was in the commit phase on the publisher.
+ * Use FullTransactionId to prevent issues with transaction ID wraparound,
+ * where a new remote_oldestxid could falsely appear to originate from the
+ * past and block advancement.
+ */
+ FullTransactionId remote_oldestxid;
+
+ /*
+ * Next transaction ID to be assigned on the publisher. Use
+ * FullTransactionId for consistency and to allow straightforward
+ * comparisons with remote_oldestxid.
+ */
+ FullTransactionId remote_nextxid;
+
+ TimestampTz reply_time; /* when the publisher responds with status */
+
+ /*
+ * Publisher transaction ID that must be awaited to complete before
+ * entering the final phase (RDT_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same reason as remote_nextxid.
+ */
+ FullTransactionId remote_wait_for;
+
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ TimestampTz flushpos_update_time; /* when the remote flush position was
+ * updated in final phase
+ * (RDT_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 */
+} RetainDeadTuplesData;
+
+/*
+ * 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 =
{
@@ -332,6 +496,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +543,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data);
+static void process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void get_candidate_xid(RetainDeadTuplesData *rdt_data);
+static void request_publisher_status(RetainDeadTuplesData *rdt_data);
+static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3761,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainDeadTuplesData rdt_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3840,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rdt_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3868,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rdt_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3885,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rdt_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rdt_data.remote_lsn = pq_getmsgint64(&s);
+ rdt_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rdt_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rdt_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(rdt_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rdt_data, true);
+
+ UpdateWorkerStats(last_received, rdt_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3922,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rdt_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rdt_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3961,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ rdt_data.xid_advance_interval)
+ wait_time = Min(wait_time, rdt_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +4032,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rdt_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +4069,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;
@@ -3920,6 +4146,363 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
last_flushpos = flushpos;
}
+/*
+ * Attempt to advance the non-removable transaction ID.
+ *
+ * See comments atop worker.c for details.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rdt_data))
+ return;
+
+ process_rdt_phase_transition(rdt_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining dead tuples is not required */
+ if (!MySubscription->retaindeadtuples)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop worker.c for details of the transition.
+ */
+static void
+process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
+ bool status_received)
+{
+ switch (rdt_data->phase)
+ {
+ case RDT_GET_CANDIDATE_XID:
+ get_candidate_xid(rdt_data);
+ break;
+ case RDT_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rdt_data);
+ break;
+ case RDT_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rdt_data, status_received);
+ break;
+ case RDT_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rdt_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RDT_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId oldest_running_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 = rdt_data->last_recv_time ? rdt_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(rdt_data->candidate_xid_time, now,
+ rdt_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rdt_data->candidate_xid_time = now;
+
+ oldest_running_xid = GetOldestActiveTransactionId(false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rdt_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rdt_data, true);
+
+ rdt_data->candidate_xid = oldest_running_xid;
+ rdt_data->phase = RDT_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainDeadTuplesData *rdt_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);
+
+ rdt_data->phase = RDT_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 RDT_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
+ rdt_data->remote_wait_for = rdt_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RDT_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rdt_data->remote_wait_for,
+ rdt_data->remote_oldestxid))
+ rdt_data->phase = RDT_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rdt_data->phase = RDT_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainDeadTuplesData *rdt_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rdt_data->remote_lsn) &&
+ TransactionIdIsValid(rdt_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop worker.c.
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rdt_data->reply_time,
+ rdt_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rdt_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rdt_data->remote_lsn && rdt_data->last_recv_time &&
+ TimestampDifferenceExceeds(rdt_data->flushpos_update_time,
+ rdt_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;
+
+ rdt_data->flushpos_update_time = rdt_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rdt_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 = rdt_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ rdt_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rdt_data->phase = RDT_GET_CANDIDATE_XID;
+ rdt_data->remote_lsn = InvalidXLogRecPtr;
+ rdt_data->remote_oldestxid = InvalidFullTransactionId;
+ rdt_data->remote_nextxid = InvalidFullTransactionId;
+ rdt_data->reply_time = 0;
+ rdt_data->remote_wait_for = InvalidFullTransactionId;
+ rdt_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_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(RetainDeadTuplesData *rdt_data, bool new_xid_found)
+{
+ if (!new_xid_found && rdt_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.
+ */
+ rdt_data->xid_advance_interval = Min(rdt_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.
+ */
+ rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5291,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_dead_tuples was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_dead_tuples as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_dead_tuples is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retaindeadtuples &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_dead_tuples"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5471,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ if (MySubscription->retaindeadtuples)
+ CheckSubDeadTupleRetention(false, true, WARNING);
+
proc_exit(0);
}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index e44ad576bc7..8605776ad86 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 IsSlotForConflictCheck(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,31 @@ ReplicationSlotValidateName(const char *name, int elevel)
return false;
}
}
+
+ if (!allow_reserved_name && IsSlotForConflictCheck(name))
+ {
+ ereport(elevel,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ return false;
+ }
+
return true;
}
+/*
+ * Return true if the replication slot name is "pg_conflict_detection".
+ */
+static bool
+IsSlotForConflictCheck(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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsSlotForConflictCheck(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 28b8591efa5..1723ab7308d 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,56 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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 = GetOldestActiveTransactionId(true);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 2418967def6..76cb0ab1de9 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2823,9 +2823,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2852,6 +2855,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2859,6 +2864,10 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index c6226175528..ca1f3f026f8 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretaindeadtuples;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretaindeadtuples\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretaindeadtuples\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5104,6 +5113,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].subretaindeadtuples =
+ (strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5362,6 +5373,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subretaindeadtuples)
+ appendPQExpBufferStr(query, ", retain_dead_tuples = 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 39eef1d6617..41b3f39cee4 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subretaindeadtuples;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/pg_upgrade/check.c b/src/bin/pg_upgrade/check.c
index 30579ef2051..5e6403f0773 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -28,7 +28,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -631,7 +631,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();
}
@@ -764,7 +764,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -2040,48 +2040,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_dead_tuples enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rdt_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rdt_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_dead_tuples option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_dead_tuples)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rdt_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_dead_tuples
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rdt_slot_on_new = PQfnumber(res, "rdt_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rdt_slot_on_new = atoi(PQgetvalue(res, 0, i_rdt_slot_on_new));
+
+ if (rdt_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_dead_tuples);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -2094,12 +2126,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_dead_tuples &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_dead_tuples &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2211,6 +2255,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..a437067cdca 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_dead_tuples;
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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretaindeadtuples THEN 1 END) > 0 AS retain_dead_tuples "
+ "FROM pg_catalog.pg_subscription");
+ else
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "'f' AS retain_dead_tuples "
+ "FROM pg_catalog.pg_subscription");
+
+ i_nsub = PQfnumber(res, "nsub");
+ i_retain_dead_tuples = PQfnumber(res, "retain_dead_tuples");
+
+ cluster->nsubs = atoi(PQgetvalue(res, 0, i_nsub));
+ cluster->sub_retain_dead_tuples = (strcmp(PQgetvalue(res, 0, i_retain_dead_tuples), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..d5cd5bf0b3a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_dead_tuples)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_dead_tuples)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_dead_tuples enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..e9401430e69 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_dead_tuples; /* whether a subscription enables
+ * retain_dead_tuples. */
} 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 e46f02c6cc6..77387be0f9d 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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_dead_tuples 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_dead_tuples = 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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_dead_tuples
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_dead_tuples = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_dead_tuples
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_dead_tuples
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretaindeadtuples set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretaindeadtuples FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_dead_tuples are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..7a06af48842 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ appendPQExpBuffer(&buf,
+ ", subretaindeadtuples AS \"%s\"\n",
+ gettext_noop("Retain dead tuples"));
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 37524364290..dbc586c5bc3 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2319,8 +2319,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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_dead_tuples",
+ "run_as_owner", "slot_name", "streaming",
+ "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3774,8 +3775,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_dead_tuples",
+ "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_proc.dat b/src/include/catalog/pg_proc.dat
index 1fc19146f46..3ee8fed7e53 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11801,6 +11801,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..231ef84ec9a 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 subretaindeadtuples; /* True if dead tuples useful for
+ * conflict detection are 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 retaindeadtuples; /* True if dead tuples useful for conflict
+ * detection are 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..9b288ad22a6 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,9 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
+extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 76aeeb92242..9020764d253 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..0c7b8440a61 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_dead_tuples);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 e4877d88e8f..7cbc8bd59bd 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -55,7 +55,7 @@ extern RunningTransactions GetRunningTransactionData(void);
extern bool TransactionIdIsInProgress(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 529b2241731..a98c97f7616 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00012345
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,34 @@ 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - retain_dead_tuples must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = foo);
+ERROR: retain_dead_tuples requires a Boolean value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = false);
+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 dead tuples | 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/00000000
(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 007c9e70374..f0f714fe747 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - retain_dead_tuples must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..7458d7fba7e 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,195 @@ $node_subscriber->wait_for_log(
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_dead_tuples = 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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_dead_tuples option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_dead_tuples option.
+##################################################
+
+# Alter retain_dead_tuples for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_dead_tuples = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_dead_tuples\" for enabled subscription/,
+ "altering retain_dead_tuples is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_dead_tuples for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_dead_tuples = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_dead_tuples is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+##################################################
+# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
+# enabled. This warns of the possibility of receiving changes from origins
+# other than the publisher.
+##################################################
+
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = any);");
+ok( $stderr =~
+ /WARNING: subscription "tap_sub_a_b" enabled retain_dead_tuples but might not reliably detect conflicts for changes from different origins/,
+ "warn of the possibility of receiving changes from origins other than the publisher");
+
+# Reset the origin to none
+$node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = none);");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index ff050e93a50..ff373c3f8cc 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2565,6 +2565,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainDeadTuplesData
+RetainDeadTuplesPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
On Sat, Jul 19, 2025 at 10:32 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Jul 19, 2025 at 3:01 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jul 18, 2025 at 5:03 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here are some review comments and questions:
--- + if (inCommitOnly && + (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0) + continue; +I've not verified yet but is it possible that we exclude XIDs of
processes who are running on other databases?I can't see how, even the comments atop function says: " We look at
all databases, though there is no need to include WALSender since this
has no effect on hot standby conflicts." which indicate that it
shouldn't exlude XIDs of procs who are running on other databases.
I think I misunderstood your question. You were asking if possible, we
should exclude XIDs of processes running on other databases in the
above check as for our purpose, we don't need those. If so, I agree
with you, we don't need XIDs of other databases as logical WALSender
will anyway won't process transactions in other databases, so we can
exclude those. The function GetOldestActiveTransactionId() is called
from two places in patch get_candidate_xid() and
ProcessStandbyPSRequestMessage(). We don't need to care for XIDs in
other databases at both places but care for
Commit_Critical_Section_Phase when called from
ProcessStandbyPSRequestMessage(). So, we probably need two parameters
to distinguish those cases.
--
With Regards,
Amit Kapila.
On Mon, Jul 21, 2025 at 9:30 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Jul 19, 2025 at 10:32 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Jul 19, 2025 at 3:01 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jul 18, 2025 at 5:03 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here are some review comments and questions:
--- + if (inCommitOnly && + (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0) + continue; +I've not verified yet but is it possible that we exclude XIDs of
processes who are running on other databases?I can't see how, even the comments atop function says: " We look at
all databases, though there is no need to include WALSender since this
has no effect on hot standby conflicts." which indicate that it
shouldn't exlude XIDs of procs who are running on other databases.I think I misunderstood your question. You were asking if possible, we
should exclude XIDs of processes running on other databases in the
above check as for our purpose, we don't need those. If so, I agree
with you, we don't need XIDs of other databases as logical WALSender
will anyway won't process transactions in other databases, so we can
exclude those. The function GetOldestActiveTransactionId() is called
from two places in patch get_candidate_xid() and
ProcessStandbyPSRequestMessage(). We don't need to care for XIDs in
other databases at both places but care for
Commit_Critical_Section_Phase when called from
ProcessStandbyPSRequestMessage(). So, we probably need two parameters
to distinguish those cases.
It seems unnecessary to track transactions on other databases, as they
won't be replicated to the subscriber.
So, a new parameter 'allDbs' is introduced to control the filtering of
transactions from other databases.
Attached updated V51 patch.
Thank you Hou-san for updating the patch for this change.
--
Thanks,
Nisha
Attachments:
v51-0001-Preserve-conflict-relevant-data-during-logical-r.patchapplication/octet-stream; name=v51-0001-Preserve-conflict-relevant-data-during-logical-r.patchDownload
From 48e142321abf3db4e911f150af6ed95ea6abbf8c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 18 Jul 2025 17:31:20 +0800
Subject: [PATCH v51] Preserve conflict-relevant data during logical
replication.
Logical replication requires reliable conflict detection to maintain data
consistency across nodes. To achieve this, we must prevent premature
removal of tuples deleted by other origins and their associated commit_ts
data by VACUUM, which could otherwise lead to incorrect conflict reporting
and resolution.
This patch introduces a mechanism to retain deleted tuples on the
subscriber during the application of concurrent transactions from remote
nodes. Retaining these tuples allows us to correctly ignore concurrent
updates to the same tuple. Without this, an UPDATE might be misinterpreted
as an INSERT during resolutions due to the absence of the original tuple.
Additionally, we ensure that origin metadata is not prematurely removed by
vacuum freeze, which is essential for detecting update_origin_differs and
delete_origin_differs conflicts.
To support this, a new replication slot named pg_conflict_detection is
created and maintained by the launcher on the subscriber. Each apply
worker tracks its own non-removable transaction ID, which the launcher
aggregates to determine the appropriate xmin for the slot, thereby
retaining necessary tuples.
Conflict information retention (deleted tuples and commit_ts) can be
enabled per subscription via the retain_dead_tuples option. This is
disabled by default to avoid unnecessary overhead for configurations that
do not require conflict resolution or logging.
During upgrades, if any subscription on the old cluster has
retain_dead_tuples enabled, a conflict detection slot will be created to
protect relevant tuples from deletion when the new cluster starts.
This is a foundational work to correctly detect update_deleted conflict
which will be done in a follow-up patch.
Bump catalog version
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Discussion: https://postgr.es/m/OS0PR01MB5716BE80DAEB0EE2A6A5D1F5949D2@OS0PR01MB5716.jpnprd01.prod.outlook.com
---
doc/src/sgml/catalogs.sgml | 11 +
doc/src/sgml/config.sgml | 2 +
doc/src/sgml/func.sgml | 16 +-
doc/src/sgml/logical-replication.sgml | 32 +
doc/src/sgml/protocol.sgml | 88 +++
doc/src/sgml/ref/alter_subscription.sgml | 18 +-
doc/src/sgml/ref/create_subscription.sgml | 87 ++-
src/backend/access/transam/twophase.c | 32 +-
src/backend/access/transam/xact.c | 18 +-
src/backend/access/transam/xlog.c | 2 +-
src/backend/access/transam/xlogrecovery.c | 2 +-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 3 +-
src/backend/commands/subscriptioncmds.c | 400 +++++++++--
.../replication/logical/applyparallelworker.c | 3 +-
src/backend/replication/logical/launcher.c | 228 ++++++-
.../replication/logical/reorderbuffer.c | 2 +-
src/backend/replication/logical/tablesync.c | 3 +-
src/backend/replication/logical/worker.c | 621 +++++++++++++++++-
src/backend/replication/slot.c | 48 +-
src/backend/replication/walsender.c | 60 ++
src/backend/storage/ipc/procarray.c | 20 +-
src/backend/utils/adt/pg_upgrade_support.c | 19 +
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/pg_upgrade/check.c | 96 ++-
src/bin/pg_upgrade/info.c | 25 +-
src/bin/pg_upgrade/pg_upgrade.c | 60 +-
src/bin/pg_upgrade/pg_upgrade.h | 4 +-
src/bin/pg_upgrade/t/004_subscription.pl | 85 ++-
src/bin/psql/describe.c | 6 +-
src/bin/psql/tab-complete.in.c | 10 +-
src/include/catalog/pg_proc.dat | 4 +
src/include/catalog/pg_subscription.h | 5 +
src/include/commands/subscriptioncmds.h | 5 +
src/include/replication/logicallauncher.h | 3 +
src/include/replication/slot.h | 11 +-
src/include/replication/worker_internal.h | 13 +-
src/include/storage/proc.h | 8 +
src/include/storage/procarray.h | 3 +-
src/test/regress/expected/subscription.out | 168 ++---
src/test/regress/sql/subscription.sql | 11 +
src/test/subscription/t/035_conflicts.pl | 195 +++++-
src/tools/pgindent/typedefs.list | 2 +
44 files changed, 2230 insertions(+), 219 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 0d23bc1b122..97f547b3cc4 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8082,6 +8082,17 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretaindeadtuples</structfield> <type>bool</type>
+ </para>
+ <para>
+ If true, the information (e.g., dead tuples, commit timestamps, and
+ origins) on the subscriber that is useful for conflict detection is
+ retained.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index c7acc0f182f..20ccb2d6b54 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -4965,6 +4965,8 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
new setting.
This setting has no effect if <varname>primary_conninfo</varname> is not
set or the server is not in standby mode.
+ The name cannot be <literal>pg_conflict_detection</literal> as it is
+ reserved for the conflict detection slot.
</para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/func.sgml b/doc/src/sgml/func.sgml
index f5a0e0954a1..de5b5929ee0 100644
--- a/doc/src/sgml/func.sgml
+++ b/doc/src/sgml/func.sgml
@@ -29592,7 +29592,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
</para>
<para>
Creates a new physical replication slot named
- <parameter>slot_name</parameter>. The optional second parameter,
+ <parameter>slot_name</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection slot. The optional second parameter,
when <literal>true</literal>, specifies that the <acronym>LSN</acronym> for this
replication slot be reserved immediately; otherwise
the <acronym>LSN</acronym> is reserved on first connection from a streaming
@@ -29636,7 +29638,9 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Creates a new logical (decoding) replication slot named
<parameter>slot_name</parameter> using the output plugin
- <parameter>plugin</parameter>. The optional third
+ <parameter>plugin</parameter>. The name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection slot. The optional third
parameter, <parameter>temporary</parameter>, when set to true, specifies that
the slot should not be permanently stored to disk and is only meant
for use by the current session. Temporary slots are also
@@ -29666,6 +29670,8 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
<para>
Copies an existing physical replication slot named <parameter>src_slot_name</parameter>
to a physical replication slot named <parameter>dst_slot_name</parameter>.
+ The new slot name cannot be <literal>pg_conflict_detection</literal>,
+ as it is reserved for the conflict detection.
The copied physical slot starts to reserve WAL from the same <acronym>LSN</acronym> as the
source slot.
<parameter>temporary</parameter> is optional. If <parameter>temporary</parameter>
@@ -29688,8 +29694,10 @@ postgres=# SELECT '0/0'::pg_lsn + pd.segment_number * ps.setting::int + :offset
Copies an existing logical replication slot
named <parameter>src_slot_name</parameter> to a logical replication
slot named <parameter>dst_slot_name</parameter>, optionally changing
- the output plugin and persistence. The copied logical slot starts
- from the same <acronym>LSN</acronym> as the source logical slot. Both
+ the output plugin and persistence. The new slot name cannot be
+ <literal>pg_conflict_detection</literal> as it is reserved for
+ the conflict detection. The copied logical slot starts from the same
+ <acronym>LSN</acronym> as the source logical slot. Both
<parameter>temporary</parameter> and <parameter>plugin</parameter> are
optional; if they are omitted, the values of the source slot are used.
The <literal>failover</literal> option of the source logical slot
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index e26f7f59d4a..65b090cb3dd 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -2396,6 +2396,12 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
the subscriber, plus some reserve for table synchronization.
</para>
+ <para>
+ <link linkend="guc-max-replication-slots"><varname>max_replication_slots</varname></link>
+ must be set to at least 1 when <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled for any subscription.
+ </para>
+
<para>
<link linkend="guc-max-logical-replication-workers"><varname>max_logical_replication_workers</varname></link>
must be set to at least the number of subscriptions (for leader apply
@@ -2532,6 +2538,22 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
dependencies on clusters before version 17.0 will silently be ignored.
</para>
+ <note>
+ <para>
+ Commit timestamps and origin data are not preserved during the upgrade.
+ As a result, even if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, the upgraded subscriber may be unable to detect conflicts or
+ log relevant commit timestamps and origins when applying changes from the
+ publisher occurred before the upgrade. Additionally, immediately after the
+ upgrade, the vacuum may remove the deleted rows that are required for
+ conflict detection. This can affect the changes that were not replicated
+ before the upgrade. To ensure consistent conflict tracking, users should
+ ensure that all potentially conflicting changes are replicated to the
+ subscriber before initiating the upgrade.
+ </para>
+ </note>
+
<para>
There are some prerequisites for <application>pg_upgrade</application> to
be able to upgrade the subscriptions. If these are not met an error
@@ -2563,6 +2585,16 @@ CONTEXT: processing remote data for replication origin "pg_16395" during "INSER
subscriptions present in the old cluster.
</para>
</listitem>
+ <listitem>
+ <para>
+ If there are subscriptions with retain_dead_tuples enabled, the reserved
+ replication slot <quote><literal>pg_conflict_detection</literal></quote>
+ must not exist on the new cluster. Additionally, the
+ <link linkend="guc-wal-level"><varname>wal_level</varname></link> on the
+ new cluster must be set to <literal>replica</literal> or
+ <literal>logical</literal>.
+ </para>
+ </listitem>
</itemizedlist>
</sect2>
diff --git a/doc/src/sgml/protocol.sgml b/doc/src/sgml/protocol.sgml
index e74b5be1eff..b115884acb3 100644
--- a/doc/src/sgml/protocol.sgml
+++ b/doc/src/sgml/protocol.sgml
@@ -2235,6 +2235,8 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
<para>
The name of the slot to create. Must be a valid replication slot
name (see <xref linkend="streaming-replication-slots-manipulation"/>).
+ The name cannot be <literal>pg_conflict_detection</literal> as it
+ is reserved for the conflict detection.
</para>
</listitem>
</varlistentry>
@@ -2653,6 +2655,65 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-primary-status-update">
+ <term>Primary status update (B)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('s')</term>
+ <listitem>
+ <para>
+ Identifies the message as a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The latest WAL write position on the server.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The oldest transaction ID that is currently in the commit phase on
+ the server, along with its epoch. The most significant 32 bits are
+ the epoch. The least significant 32 bits are the transaction ID.
+ If no transactions are active on the server, this number will be
+ the next transaction ID to be assigned.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The next transaction ID to be assigned on the server, along with
+ its epoch. The most significant 32 bits are the epoch. The least
+ significant 32 bits are the transaction ID.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The server's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
</variablelist>
<para>
@@ -2797,6 +2858,33 @@ psql "dbname=postgres replication=database" -c "IDENTIFY_SYSTEM;"
</variablelist>
</listitem>
</varlistentry>
+
+ <varlistentry id="protocol-replication-standby-wal-status-request">
+ <term>Request primary status update (F)</term>
+ <listitem>
+ <variablelist>
+ <varlistentry>
+ <term>Byte1('p')</term>
+ <listitem>
+ <para>
+ Identifies the message as a request for a primary status update.
+ </para>
+ </listitem>
+ </varlistentry>
+
+ <varlistentry>
+ <term>Int64</term>
+ <listitem>
+ <para>
+ The client's system clock at the time of transmission, as
+ microseconds since midnight on 2000-01-01.
+ </para>
+ </listitem>
+ </varlistentry>
+ </variablelist>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index fdc648d007f..d48cdc76bd3 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -235,8 +235,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-password-required"><literal>password_required</literal></link>,
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
- <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>, and
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>.
+ <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
@@ -261,8 +262,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
</para>
<para>
- The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>
- and <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>
+ The <link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
parameters can only be altered when the subscription is disabled.
</para>
@@ -285,6 +287,14 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
option is changed from <literal>true</literal> to <literal>false</literal>,
the publisher will replicate the transactions again when they are committed.
</para>
+
+ <para>
+ If the <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ option is altered to <literal>false</literal> and no other subscription
+ has this option enabled, the replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote>, created to retain
+ dead tuples for conflict detection, will be dropped.
+ </para>
</listitem>
</varlistentry>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 57dec28a5df..b8cd15f3280 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -169,7 +169,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<listitem>
<para>
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 <literal>pg_conflict_detection</literal> as it is reserved for the
+ conflict detection.
</para>
<para>
@@ -435,6 +437,89 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-retain-dead-tuples">
+ <term><literal>retain_dead_tuples</literal> (<type>boolean</type>)</term>
+ <listitem>
+ <para>
+ Specifies whether the information (e.g., dead tuples, commit
+ timestamps, and origins) required for conflict detection on the
+ subscriber is retained. The default is <literal>false</literal>.
+ If set to <literal>true</literal>, a physical replication slot named
+ <quote><literal>pg_conflict_detection</literal></quote> will be
+ created on the subscriber to prevent the conflict information from
+ being removed.
+ </para>
+
+ <para>
+ Note that the information useful for conflict detection is retained
+ only after the creation of the slot. You can verify the existence of
+ this slot by querying <link linkend="view-pg-replication-slots">pg_replication_slots</link>.
+ And even if multiple subscriptions on one node enable this option,
+ only one replication slot will be created. Also,
+ <varname>wal_level</varname> must be set to <literal>replica</literal>
+ or higher to allow the replication slot to be used.
+ </para>
+
+ <caution>
+ <para>
+ Note that the information for conflict detection cannot be purged if
+ the subscription is disabled; thus, the information will accumulate
+ until the subscription is enabled. To prevent excessive accumulation,
+ it is recommended to disable <literal>retain_dead_tuples</literal>
+ if the subscription will be inactive for an extended period.
+ </para>
+
+ <para>
+ Additionally when enabling <literal>retain_dead_tuples</literal> for
+ conflict detection in logical replication, it is important to design the
+ replication topology to balance data retention requirements with
+ overall system performance. This option provides minimal performance
+ overhead when applied appropriately. The following scenarios illustrate
+ effective usage patterns when enabling this option.
+ </para>
+
+ <para>
+ a. Large Tables with Bidirectional Writes:
+ For large tables subject to concurrent writes on both publisher and
+ subscriber nodes, publishers can define row filters when creating
+ publications to segment data. This allows multiple subscriptions
+ to replicate exclusive subsets of the table in parallel, optimizing
+ the throughput.
+ </para>
+
+ <para>
+ b. Write-Enabled Subscribers:
+ If a subscriber node is expected to perform write operations, replication
+ can be structured using multiple publications and subscriptions. By
+ distributing tables across these publications, the workload is spread among
+ several apply workers, improving concurrency and reducing contention.
+ </para>
+
+ <para>
+ c. Read-Only Subscribers:
+ In configurations involving single or multiple publisher nodes
+ performing concurrent write operations, read-only subscriber nodes may
+ replicate changes without seeing a performance impact if it does index
+ scan. However, if the subscriber is impacted due to replication lag or
+ scan performance (say due to sequential scans), it needs to follow one
+ of the two previous strategies to distribute the workload on the
+ subscriber.
+ </para>
+ </caution>
+
+ <para>
+ This option cannot be enabled if the publisher is a physical standby.
+ </para>
+
+ <para>
+ Enabling this option ensures retention of information useful for
+ conflict detection solely for changes occurring locally on the
+ publisher. For the changes originating from different origins,
+ reliable conflict detection cannot be guaranteed.
+ </para>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 85cbe397cb2..7918176fc58 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -1183,7 +1183,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 in logical replication. See
+ * GetOldestActiveTransactionId(true, false) and its use.
*
* We save the PREPARE record's location in the gxact for later use by
* CheckPointTwoPhase.
@@ -2298,7 +2302,7 @@ ProcessTwoPhaseBuffer(FullTransactionId fxid,
* 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),
@@ -2318,7 +2322,7 @@ RecordTransactionCommitPrepared(TransactionId xid,
const char *gid)
{
XLogRecPtr recptr;
- TimestampTz committs = GetCurrentTimestamp();
+ TimestampTz committs;
bool replorigin;
/*
@@ -2331,8 +2335,24 @@ 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;
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
+ * commit time is written.
+ */
+ pg_write_barrier();
+
+ /*
+ * 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 transactions that have acquired commit timestamp
+ * 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 comments atop worker.c.
+ */
+ committs = GetCurrentTimestamp();
/*
* Emit the XLOG commit record. Note that we mark 2PC commits as
@@ -2381,7 +2401,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 41601fcb280..b46e7e9c2a6 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -1431,10 +1431,22 @@ RecordTransactionCommit(void)
* without holding the ProcArrayLock, since we're the only one
* modifying it. This makes checkpoint's determination of which xacts
* are delaying the checkpoint a bit fuzzy, but it doesn't matter.
+ *
+ * Note, it is important to get the commit timestamp after marking the
+ * transaction in the commit critical section. See
+ * RecordTransactionCommitPrepared.
*/
- 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;
+
+ Assert(xactStopTimestamp == 0);
+
+ /*
+ * Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible
+ * before commit time is written.
+ */
+ pg_write_barrier();
/*
* Insert the commit XLOG record.
@@ -1537,7 +1549,7 @@ RecordTransactionCommit(void)
*/
if (markXidCommitted)
{
- MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
+ MyProc->delayChkptFlags &= ~DELAY_CHKPT_IN_COMMIT;
END_CRIT_SECTION();
}
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 8e7827c6ed9..eefffc4277a 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -7121,7 +7121,7 @@ CreateCheckPoint(int flags)
* starting snapshot of locks and transactions.
*/
if (!shutdown && XLogStandbyInfoActive())
- checkPoint.oldestActiveXid = GetOldestActiveTransactionId();
+ checkPoint.oldestActiveXid = GetOldestActiveTransactionId(false, true);
else
checkPoint.oldestActiveXid = InvalidTransactionId;
diff --git a/src/backend/access/transam/xlogrecovery.c b/src/backend/access/transam/xlogrecovery.c
index 23878b2dd91..e8f3ba00caa 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/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 1395032413e..63c2992d19f 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->retaindeadtuples = subform->subretaindeadtuples;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index b2d5332effc..f6eca09ee15 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1386,7 +1386,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)
+ subretaindeadtuples, 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 e23b0de7242..cd6c3684482 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_DEAD_TUPLES 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 retaindeadtuples;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -105,8 +108,10 @@ typedef struct SubOpts
static List *fetch_table_list(WalReceiverConn *wrconn, List *publications);
static void check_publications_origin(WalReceiverConn *wrconn,
List *publications, bool copydata,
- char *origin, Oid *subrel_local_oids,
- int subrel_count, char *subname);
+ bool retain_dead_tuples, char *origin,
+ Oid *subrel_local_oids, int subrel_count,
+ char *subname);
+static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
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_DEAD_TUPLES))
+ opts->retaindeadtuples = false;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -210,7 +217,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)
@@ -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_DEAD_TUPLES) &&
+ strcmp(defel->defname, "retain_dead_tuples") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
+ opts->retaindeadtuples = 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_DEAD_TUPLES | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -630,6 +647,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
+ /* Ensure that we can enable retain_dead_tuples */
+ if (opts.retaindeadtuples)
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -670,6 +691,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_subretaindeadtuples - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -722,7 +745,11 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
{
check_publications(wrconn, publications);
check_publications_origin(wrconn, publications, opts.copy_data,
- opts.origin, NULL, 0, stmt->subname);
+ opts.retaindeadtuples, opts.origin,
+ NULL, 0, stmt->subname);
+
+ if (opts.retaindeadtuples)
+ check_pub_dead_tuple_retention(wrconn);
/*
* Set sync state based on if we were asked to do data copy or
@@ -881,8 +908,8 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
sizeof(Oid), oid_cmp);
check_publications_origin(wrconn, sub->publications, copy_data,
- sub->origin, subrel_local_oids,
- subrel_count, sub->name);
+ sub->retaindeadtuples, sub->origin,
+ subrel_local_oids, subrel_count, sub->name);
/*
* Rels that we want to remove from subscription and drop any slots
@@ -1040,18 +1067,22 @@ AlterSubscription_refresh(Subscription *sub, bool copy_data,
}
/*
- * Common checks for altering failover and two_phase options.
+ * Common checks for altering failover, two_phase, and retain_dead_tuples
+ * options.
*/
static void
CheckAlterSubOption(Subscription *sub, const char *option,
bool slot_needs_update, bool isTopLevel)
{
+ Assert(strcmp(option, "failover") == 0 ||
+ strcmp(option, "two_phase") == 0 ||
+ strcmp(option, "retain_dead_tuples") == 0);
+
/*
- * The checks in this function are required only for failover and
- * two_phase options.
+ * Altering the retain_dead_tuples option does not update the slot on the
+ * publisher.
*/
- Assert(strcmp(option, "failover") == 0 ||
- strcmp(option, "two_phase") == 0);
+ Assert(!slot_needs_update || strcmp(option, "retain_dead_tuples") != 0);
/*
* Do not allow changing the option if the subscription is enabled. This
@@ -1063,6 +1094,39 @@ CheckAlterSubOption(Subscription *sub, const char *option,
* the publisher by the existing walsender, so we could have allowed that
* even when the subscription is enabled. But we kept this restriction for
* the sake of consistency and simplicity.
+ *
+ * Additionally, do not allow changing the retain_dead_tuples option when
+ * the subscription is enabled to prevent race conditions arising from the
+ * new option value being acknowledged asynchronously by the launcher and
+ * apply workers.
+ *
+ * Without the restriction, a race condition may arise when a user
+ * disables and immediately re-enables the retain_dead_tuples option. In
+ * this case, the launcher might drop the slot upon noticing the disabled
+ * action, while the apply worker may keep maintaining
+ * oldest_nonremovable_xid without noticing the option change. During this
+ * period, a transaction ID wraparound could falsely make this ID appear
+ * as if it originates from the future w.r.t the transaction ID stored in
+ * the slot maintained by launcher.
+ *
+ * Similarly, if the user enables retain_dead_tuples concurrently with the
+ * launcher starting the worker, the apply worker may start calculating
+ * oldest_nonremovable_xid before the launcher notices the enable action.
+ * Consequently, the launcher may update slot.xmin to a newer value than
+ * that maintained by the worker. In subsequent cycles, upon integrating
+ * the worker's oldest_nonremovable_xid, the launcher might detect a
+ * retreat in the calculated xmin, necessitating additional handling.
+ *
+ * XXX To address the above race conditions, we can define
+ * oldest_nonremovable_xid as FullTransactionID and adds the check to
+ * disallow retreating the conflict slot's xmin. For now, we kept the
+ * implementation simple by disallowing change to the retain_dead_tuples,
+ * but in the future we can change this after some more analysis.
+ *
+ * Note that we could restrict only the enabling of retain_dead_tuples to
+ * avoid the race conditions described above, but we maintain the
+ * restriction for both enable and disable operations for the sake of
+ * consistency.
*/
if (sub->enabled)
ereport(ERROR,
@@ -1110,6 +1174,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_tuple = false;
bool update_failover = false;
bool update_two_phase = false;
+ bool check_pub_rdt = false;
+ bool retain_dead_tuples;
+ char *origin;
Subscription *sub;
Form_pg_subscription form;
bits32 supported_opts;
@@ -1137,6 +1204,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
sub = GetSubscription(subid, false);
+ retain_dead_tuples = sub->retaindeadtuples;
+ origin = sub->origin;
+
/*
* Don't allow non-superuser modification of a subscription with
* password_required=false.
@@ -1165,7 +1235,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1325,11 +1395,62 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
replaces[Anum_pg_subscription_subfailover - 1] = true;
}
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
+ {
+ values[Anum_pg_subscription_subretaindeadtuples - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+
+ CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
+
+ /*
+ * Workers may continue running even after the
+ * subscription has been disabled.
+ *
+ * To prevent race conditions (as described in
+ * CheckAlterSubOption()), ensure that all worker
+ * processes have already exited before proceeding.
+ */
+ if (logicalrep_workers_find(subid, true, true))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
+ errhint("Try again after some time.")));
+
+ /*
+ * Remind the user that enabling subscription will prevent
+ * the accumulation of dead tuples.
+ */
+ if (opts.retaindeadtuples)
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
+
+ /*
+ * 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();
+
+ check_pub_rdt = opts.retaindeadtuples;
+ retain_dead_tuples = opts.retaindeadtuples;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
CStringGetTextDatum(opts.origin);
replaces[Anum_pg_subscription_suborigin - 1] = true;
+
+ /*
+ * Check if changes from different origins may be received
+ * from the publisher when the origin is changed to ANY
+ * and retain_dead_tuples is enabled.
+ */
+ check_pub_rdt = retain_dead_tuples &&
+ pg_strcasecmp(opts.origin, LOGICALREP_ORIGIN_ANY) == 0;
+
+ origin = opts.origin;
}
update_tuple = true;
@@ -1347,6 +1468,15 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("cannot enable subscription that does not have a slot name")));
+ /*
+ * Check track_commit_timestamp only when enabling the
+ * subscription in case it was disabled after creation. See
+ * comments atop CheckSubDeadTupleRetention() for details.
+ */
+ if (sub->retaindeadtuples)
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING);
+
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
replaces[Anum_pg_subscription_subenabled - 1] = true;
@@ -1355,6 +1485,14 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
ApplyLauncherWakeupAtCommit();
update_tuple = true;
+
+ /*
+ * The subscription might be initially created with
+ * connect=false and retain_dead_tuples=true, meaning the
+ * remote server's status may not be checked. Ensure this
+ * check is conducted now.
+ */
+ check_pub_rdt = sub->retaindeadtuples && opts.enabled;
break;
}
@@ -1369,6 +1507,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_dead_tuples.
+ */
+ check_pub_rdt = sub->retaindeadtuples;
break;
case ALTER_SUBSCRIPTION_SET_PUBLICATION:
@@ -1568,14 +1713,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_dead_tuples.
*
* 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 || check_pub_rdt)
{
bool must_use_password;
char *err;
@@ -1584,10 +1730,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 +1746,17 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
PG_TRY();
{
- walrcv_alter_slot(wrconn, sub->slotname,
- update_failover ? &opts.failover : NULL,
- update_two_phase ? &opts.twophase : NULL);
+ if (retain_dead_tuples)
+ check_pub_dead_tuple_retention(wrconn);
+
+ check_publications_origin(wrconn, sub->publications, false,
+ retain_dead_tuples, origin, NULL, 0,
+ sub->name);
+
+ 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();
{
@@ -2086,20 +2244,29 @@ AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId)
* Check and log a warning if the publisher has subscribed to the same table,
* its partition ancestors (if it's a partition), or its partition children (if
* it's a partitioned table), from some other publishers. This check is
- * required only if "copy_data = true" and "origin = none" for CREATE
- * SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements to notify the
- * user that data having origin might have been copied.
+ * required in the following scenarios:
*
- * This check need not be performed on the tables that are already added
- * because incremental sync for those tables will happen through WAL and the
- * origin of the data can be identified from the WAL records.
+ * 1) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "copy_data = true" and "origin = none":
+ * - Warn the user that data with an origin might have been copied.
+ * - This check is skipped for tables already added, as incremental sync via
+ * WAL allows origin tracking. The list of such tables is in
+ * subrel_local_oids.
*
- * subrel_local_oids contains the list of relation oids that are already
- * present on the subscriber.
+ * 2) For CREATE SUBSCRIPTION and ALTER SUBSCRIPTION ... REFRESH statements
+ * with "retain_dead_tuples = true" and "origin = any", and for ALTER
+ * SUBSCRIPTION statements that modify retain_dead_tuples or origin, or
+ * when the publisher's status changes (e.g., due to a connection string
+ * update):
+ * - Warn the user that only conflict detection info for local changes on
+ * the publisher is retained. Data from other origins may lack sufficient
+ * details for reliable conflict detection.
+ * - See comments atop worker.c for more details.
*/
static void
check_publications_origin(WalReceiverConn *wrconn, List *publications,
- bool copydata, char *origin, Oid *subrel_local_oids,
+ bool copydata, bool retain_dead_tuples,
+ char *origin, Oid *subrel_local_oids,
int subrel_count, char *subname)
{
WalRcvExecResult *res;
@@ -2108,9 +2275,29 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
Oid tableRow[1] = {TEXTOID};
List *publist = NIL;
int i;
+ bool check_rdt;
+ bool check_table_sync;
+ bool origin_none = origin &&
+ pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) == 0;
+
+ /*
+ * Enable retain_dead_tuples checks only when origin is set to 'any',
+ * since with origin='none' only local changes are replicated to the
+ * subscriber.
+ */
+ check_rdt = retain_dead_tuples && !origin_none;
+
+ /*
+ * Enable table synchronization checks only when origin is 'none', to
+ * ensure that data from other origins is not inadvertently copied.
+ */
+ check_table_sync = copydata && origin_none;
- if (!copydata || !origin ||
- (pg_strcasecmp(origin, LOGICALREP_ORIGIN_NONE) != 0))
+ /* retain_dead_tuples and table sync checks occur separately */
+ Assert(!(check_rdt && check_table_sync));
+
+ /* Return if no checks are required */
+ if (!check_rdt && !check_table_sync)
return;
initStringInfo(&cmd);
@@ -2129,16 +2316,23 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
/*
* In case of ALTER SUBSCRIPTION ... REFRESH, subrel_local_oids contains
* the list of relation oids that are already present on the subscriber.
- * This check should be skipped for these tables.
+ * This check should be skipped for these tables if checking for table
+ * sync scenario. However, when handling the retain_dead_tuples scenario,
+ * ensure all tables are checked, as some existing tables may now include
+ * changes from other origins due to newly created subscriptions on the
+ * publisher.
*/
- for (i = 0; i < subrel_count; i++)
+ if (check_table_sync)
{
- Oid relid = subrel_local_oids[i];
- char *schemaname = get_namespace_name(get_rel_namespace(relid));
- char *tablename = get_rel_name(relid);
+ for (i = 0; i < subrel_count; i++)
+ {
+ Oid relid = subrel_local_oids[i];
+ char *schemaname = get_namespace_name(get_rel_namespace(relid));
+ char *tablename = get_rel_name(relid);
- appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
- schemaname, tablename);
+ appendStringInfo(&cmd, "AND NOT (N.nspname = '%s' AND C.relname = '%s')\n",
+ schemaname, tablename);
+ }
}
res = walrcv_exec(wrconn, cmd.data, 1, tableRow);
@@ -2173,22 +2367,37 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
* XXX: For simplicity, we don't check whether the table has any data or
* not. If the table doesn't have any data then we don't need to
* distinguish between data having origin and data not having origin so we
- * can avoid logging a warning in that case.
+ * can avoid logging a warning for table sync scenario.
*/
if (publist)
{
StringInfo pubnames = makeStringInfo();
+ StringInfo err_msg = makeStringInfo();
+ StringInfo err_hint = makeStringInfo();
/* Prepare the list of publication(s) for warning message. */
GetPublicationsStr(publist, pubnames, false);
+
+ if (check_table_sync)
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin"),
+ subname);
+ appendStringInfoString(err_hint, _("Verify that initial data copied from the publisher tables did not come from other origins."));
+ }
+ else
+ {
+ appendStringInfo(err_msg, _("subscription \"%s\" enabled retain_dead_tuples but might not reliably detect conflicts for changes from different origins"),
+ subname);
+ appendStringInfoString(err_hint, _("Consider using origin = NONE or disabling retain_dead_tuples."));
+ }
+
ereport(WARNING,
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("subscription \"%s\" requested copy_data with origin = NONE but might copy data that had a different origin",
- subname),
- errdetail_plural("The subscription being created subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
- "The subscription being created subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
+ errmsg_internal("%s", err_msg->data),
+ errdetail_plural("The subscription subscribes to a publication (%s) that contains tables that are written to by other subscriptions.",
+ "The subscription subscribes to publications (%s) that contain tables that are written to by other subscriptions.",
list_length(publist), pubnames->data),
- errhint("Verify that initial data copied from the publisher tables did not come from other origins."));
+ errhint_internal("%s", err_hint->data));
}
ExecDropSingleTupleTableSlot(slot);
@@ -2196,6 +2405,101 @@ check_publications_origin(WalReceiverConn *wrconn, List *publications,
walrcv_clear_result(res);
}
+/*
+ * Determine whether the retain_dead_tuples can be enabled based on the
+ * publisher's status.
+ *
+ * This option is disallowed if the publisher is running a version earlier
+ * than the PG19, or if the publisher is in recovery (i.e., it is a standby
+ * server).
+ *
+ * See comments atop worker.c for a detailed explanation.
+ */
+static void
+check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
+{
+ WalRcvExecResult *res;
+ Oid RecoveryRow[1] = {BOOLOID};
+ TupleTableSlot *slot;
+ bool isnull;
+ bool remote_in_recovery;
+
+ if (walrcv_server_version(wrconn) < 19000)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("cannot enable retain_dead_tuples if the publisher is running a version earlier than PostgreSQL 19"));
+
+ 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_dead_tuples if the publisher is in recovery."));
+
+ ExecDropSingleTupleTableSlot(slot);
+
+ walrcv_clear_result(res);
+}
+
+/*
+ * Check if the subscriber's configuration is adequate to enable the
+ * retain_dead_tuples option.
+ *
+ * Issue an ERROR if the wal_level does not support the use of replication
+ * slots when check_guc is set to true.
+ *
+ * Issue a WARNING if track_commit_timestamp is not enabled when check_guc is
+ * set to true. This is only to highlight the importance of enabling
+ * track_commit_timestamp instead of catching all the misconfigurations, as
+ * this setting can be adjusted after subscription creation. Without it, the
+ * apply worker will simply skip conflict detection.
+ *
+ * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
+ * ERROR since users can only modify retain_dead_tuples for disabled
+ * subscriptions. And as long as the subscription is enabled promptly, it will
+ * not pose issues.
+ */
+void
+CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
+ int elevel_for_sub_disabled)
+{
+ Assert(elevel_for_sub_disabled == NOTICE ||
+ elevel_for_sub_disabled == WARNING);
+
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+}
+
/*
* Get the list of tables which belong to specified publications on the
* publisher connection.
diff --git a/src/backend/replication/logical/applyparallelworker.c b/src/backend/replication/logical/applyparallelworker.c
index d25085d3515..1fa931a7422 100644
--- a/src/backend/replication/logical/applyparallelworker.c
+++ b/src/backend/replication/logical/applyparallelworker.c
@@ -441,7 +441,8 @@ pa_launch_parallel_worker(void)
MySubscription->name,
MyLogicalRepWorker->userid,
InvalidOid,
- dsm_segment_handle(winfo->dsm_seg));
+ dsm_segment_handle(winfo->dsm_seg),
+ false);
if (launched)
{
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 4aed0dfcebb..742d9ba68e9 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static bool acquire_conflict_slot_if_exists(void);
+static void advance_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -148,6 +151,7 @@ get_subscription_list(void)
sub->owner = subform->subowner;
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
+ sub->retaindeadtuples = subform->subretaindeadtuples;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -309,7 +313,8 @@ logicalrep_workers_find(Oid subid, bool only_running, bool acquire_lock)
bool
logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname, Oid userid,
- Oid relid, dsm_handle subworker_dsm)
+ Oid relid, dsm_handle subworker_dsm,
+ bool retain_dead_tuples)
{
BackgroundWorker bgw;
BackgroundWorkerHandle *bgw_handle;
@@ -328,10 +333,13 @@ logicalrep_worker_launch(LogicalRepWorkerType wtype,
* - must be valid worker type
* - tablesync workers are only ones to have relid
* - parallel apply worker is the only kind of subworker
+ * - The replication slot used in conflict detection is created when
+ * retain_dead_tuples is enabled
*/
Assert(wtype != WORKERTYPE_UNKNOWN);
Assert(is_tablesync_worker == OidIsValid(relid));
Assert(is_parallel_apply_worker == (subworker_dsm != DSM_HANDLE_INVALID));
+ Assert(!retain_dead_tuples || MyReplicationSlot);
ereport(DEBUG1,
(errmsg_internal("starting logical replication worker for subscription \"%s\"",
@@ -454,6 +462,9 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->oldest_nonremovable_xid = retain_dead_tuples
+ ? MyReplicationSlot->data.xmin
+ : InvalidTransactionId;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1118,7 +1129,10 @@ ApplyLauncherWakeupAtCommit(void)
on_commit_launcher_wakeup = true;
}
-static void
+/*
+ * Wakeup the launcher immediately.
+ */
+void
ApplyLauncherWakeup(void)
{
if (LogicalRepCtx->launcher_pid != 0)
@@ -1150,6 +1164,12 @@ ApplyLauncherMain(Datum main_arg)
*/
BackgroundWorkerInitializeConnection(NULL, NULL, 0);
+ /*
+ * Acquire the conflict detection slot at startup to ensure it can be
+ * dropped if no longer needed after a restart.
+ */
+ acquire_conflict_slot_if_exists();
+
/* Enter main loop */
for (;;)
{
@@ -1159,6 +1179,9 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
+ bool can_advance_xmin = true;
+ bool retain_dead_tuples = false;
+ TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1168,7 +1191,14 @@ ApplyLauncherMain(Datum main_arg)
ALLOCSET_DEFAULT_SIZES);
oldctx = MemoryContextSwitchTo(subctx);
- /* Start any missing workers for enabled subscriptions. */
+ /*
+ * Start any missing workers for enabled subscriptions.
+ *
+ * Also, during the iteration through all subscriptions, we compute
+ * the minimum XID required to protect deleted tuples for conflict
+ * detection if one of the subscription enables retain_dead_tuples
+ * option.
+ */
sublist = get_subscription_list();
foreach(lc, sublist)
{
@@ -1178,6 +1208,38 @@ ApplyLauncherMain(Datum main_arg)
TimestampTz now;
long elapsed;
+ if (sub->retaindeadtuples)
+ {
+ retain_dead_tuples = true;
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions
+ * with retain_dead_tuples are enabled. This is required to
+ * ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ can_advance_xmin &= sub->enabled;
+
+ /*
+ * Create a replication slot to retain information necessary
+ * for conflict detection such as dead tuples, commit
+ * timestamps, and origins.
+ *
+ * The slot is created before starting the apply worker to
+ * prevent it from unnecessarily maintaining its
+ * oldest_nonremovable_xid.
+ *
+ * The slot is created even for a disabled subscription to
+ * ensure that conflict-related information is available when
+ * applying remote changes that occurred before the
+ * subscription was enabled.
+ */
+ CreateConflictDetectionSlot();
+ }
+
if (!sub->enabled)
continue;
@@ -1186,7 +1248,27 @@ ApplyLauncherMain(Datum main_arg)
LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
- continue; /* worker is running already */
+ {
+ /*
+ * Compute the minimum xmin required to protect dead tuples
+ * required for conflict detection among all running apply
+ * workers that enables retain_dead_tuples.
+ */
+ if (sub->retaindeadtuples && can_advance_xmin)
+ compute_min_nonremovable_xid(w, &xmin);
+
+ /* worker is running already */
+ continue;
+ }
+
+ /*
+ * Can't advance xmin of the slot unless all the workers
+ * corresponding to subscriptions with retain_dead_tuples are
+ * running, disabling the further computation of the minimum
+ * nonremovable xid.
+ */
+ if (sub->retaindeadtuples)
+ can_advance_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1210,7 +1292,8 @@ ApplyLauncherMain(Datum main_arg)
if (!logicalrep_worker_launch(WORKERTYPE_APPLY,
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
- DSM_HANDLE_INVALID))
+ DSM_HANDLE_INVALID,
+ sub->retaindeadtuples))
{
/*
* We get here either if we failed to launch a worker
@@ -1230,6 +1313,20 @@ ApplyLauncherMain(Datum main_arg)
}
}
+ /*
+ * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
+ * that requires us to retain dead tuples. Otherwise, if required,
+ * advance the slot's xmin to protect dead tuples required for the
+ * conflict detection.
+ */
+ if (MyReplicationSlot)
+ {
+ if (!retain_dead_tuples)
+ ReplicationSlotDropAcquired();
+ else if (can_advance_xmin)
+ advance_conflict_slot_xmin(xmin);
+ }
+
/* Switch back to original memory context. */
MemoryContextSwitchTo(oldctx);
/* Clean the temporary memory. */
@@ -1257,6 +1354,125 @@ ApplyLauncherMain(Datum main_arg)
/* Not reachable */
}
+/*
+ * Determine the minimum non-removable transaction ID across all apply workers
+ * for subscriptions that have retain_dead_tuples enabled. Store the result
+ * in *xmin.
+ */
+static void
+compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+{
+ TransactionId nonremovable_xid;
+
+ Assert(worker != NULL);
+
+ /*
+ * The replication slot for conflict detection must be created before the
+ * worker starts.
+ */
+ Assert(MyReplicationSlot);
+
+ SpinLockAcquire(&worker->relmutex);
+ nonremovable_xid = worker->oldest_nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ if (!TransactionIdIsValid(*xmin) ||
+ TransactionIdPrecedes(nonremovable_xid, *xmin))
+ *xmin = nonremovable_xid;
+}
+
+/*
+ * Acquire the replication slot used to retain information for conflict
+ * detection, if it exists.
+ *
+ * Return true if successfully acquired, otherwise return false.
+ */
+static bool
+acquire_conflict_slot_if_exists(void)
+{
+ if (!SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true))
+ return false;
+
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+ return true;
+}
+
+/*
+ * Advance the xmin the replication slot used to retain information required
+ * for conflict detection.
+ */
+static void
+advance_conflict_slot_xmin(TransactionId new_xmin)
+{
+ Assert(MyReplicationSlot);
+ Assert(TransactionIdIsValid(new_xmin));
+ Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+
+ /* Return if the xmin value of the slot cannot be advanced */
+ if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
+ return;
+
+ SpinLockAcquire(&MyReplicationSlot->mutex);
+ MyReplicationSlot->effective_xmin = new_xmin;
+ MyReplicationSlot->data.xmin = new_xmin;
+ SpinLockRelease(&MyReplicationSlot->mutex);
+
+ elog(DEBUG1, "updated xmin: %u", MyReplicationSlot->data.xmin);
+
+ ReplicationSlotMarkDirty();
+ ReplicationSlotsComputeRequiredXmin(false);
+
+ /*
+ * Like PhysicalConfirmReceivedLocation(), do not save slot information
+ * each time. This is acceptable because all concurrent transactions on
+ * the publisher that require the data preceding the slot's xmin should
+ * have already been applied and flushed on the subscriber before the xmin
+ * is advanced. So, even if the slot's xmin regresses after a restart, it
+ * will be advanced again in the next cycle. Therefore, no data required
+ * for conflict detection will be prematurely removed.
+ */
+ return;
+}
+
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ TransactionId xmin_horizon;
+
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ 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();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/reorderbuffer.c b/src/backend/replication/logical/reorderbuffer.c
index 7b4e8629553..5febd154b6b 100644
--- a/src/backend/replication/logical/reorderbuffer.c
+++ b/src/backend/replication/logical/reorderbuffer.c
@@ -4917,7 +4917,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/tablesync.c b/src/backend/replication/logical/tablesync.c
index e4fd6347fd1..3fea0a0206e 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -615,7 +615,8 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
MySubscription->name,
MyLogicalRepWorker->userid,
rstate->relid,
- DSM_HANDLE_INVALID);
+ DSM_HANDLE_INVALID,
+ false);
}
}
}
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c5fb627aa56..f083952c7eb 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -132,6 +132,94 @@
* failover = true when creating the subscription. Enabling failover allows us
* to smoothly transition to the promoted standby, ensuring that we can
* subscribe to the new primary without losing any data.
+ *
+ * RETAIN DEAD TUPLES
+ * ----------------------
+ * Each apply worker that enabled retain_dead_tuples option maintains a
+ * non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
+ * prevent dead rows from being removed prematurely when the apply worker still
+ * needs them to detect conflicts reliably. This helps to retain the required
+ * commit_ts module information, which further helps to detect
+ * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * otherwise, vacuum freeze could remove the required information.
+ *
+ * The logical replication launcher manages an internal replication slot named
+ * "pg_conflict_detection". It asynchronously aggregates the non-removable
+ * transaction ID from all apply workers to determine the appropriate xmin for
+ * the slot, thereby retaining necessary tuples.
+ *
+ * The non-removable transaction ID in the apply worker is advanced to the
+ * oldest running transaction ID once all concurrent transactions on the
+ * publisher have been applied and flushed locally. The process involves:
+ *
+ * - RDT_GET_CANDIDATE_XID:
+ * Call GetOldestActiveTransactionId() to take oldestRunningXid as the
+ * candidate xid.
+ *
+ * - RDT_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.
+ *
+ * - RDT_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 (RDT_REQUEST_PUBLISHER_STATUS)
+ * until all these transactions have completed.
+ *
+ * - RDT_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, detecting concurrent remote
+ * transactions with earlier timestamps than the DELETE 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.
+ *
+ * Similarly, when the publisher has subscribed to another publisher,
+ * information necessary for conflict detection cannot be retained for
+ * changes from origins other than the publisher. This is because the publisher
+ * lacks the transaction status of other publishers it subscribes to.
+ * Consequently, the non-removable transaction ID might be advanced prematurely
+ * before changes from other origins have been fully applied.
+ *
+ * XXX Retaining information for changes from other origins might be possible
+ * by requesting the subscription on that origin to enable retain_dead_tuples
+ * and fetching the conflict detection slot.xmin along with the publisher's
+ * status. In the RDT_WAIT_FOR_PUBLISHER_STATUS phase, the apply worker could
+ * wait for the remote slot's xmin to reach the oldest active transaction ID,
+ * ensuring that all transactions from other origins have been applied on the
+ * publisher, thereby getting the latest WAL position that includes all
+ * concurrent changes. However, this approach may impact performance, so it
+ * might not worth the effort.
+ *
+ * XXX It seems feasible to get the latest commit's WAL location from the
+ * publisher and wait till that is applied. However, we can't do that
+ * because commit timestamps can regress as a commit with a later LSN is not
+ * guaranteed to have a later timestamp than those with earlier LSNs. Having
+ * said that, even if that is possible, it won't improve performance much as
+ * the apply always lag and moves slowly as compared with the transactions
+ * on the publisher.
*-------------------------------------------------------------------------
*/
@@ -140,6 +228,7 @@
#include <sys/stat.h>
#include <unistd.h>
+#include "access/commit_ts.h"
#include "access/table.h"
#include "access/tableam.h"
#include "access/twophase.h"
@@ -148,6 +237,7 @@
#include "catalog/pg_inherits.h"
#include "catalog/pg_subscription.h"
#include "catalog/pg_subscription_rel.h"
+#include "commands/subscriptioncmds.h"
#include "commands/tablecmds.h"
#include "commands/trigger.h"
#include "executor/executor.h"
@@ -166,12 +256,14 @@
#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"
#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"
@@ -268,6 +360,78 @@ typedef enum
TRANS_PARALLEL_APPLY,
} TransApplyAction;
+/*
+ * The phases involved in advancing the non-removable transaction ID.
+ *
+ * See comments atop worker.c for details of the transition between these
+ * phases.
+ */
+typedef enum
+{
+ RDT_GET_CANDIDATE_XID,
+ RDT_REQUEST_PUBLISHER_STATUS,
+ RDT_WAIT_FOR_PUBLISHER_STATUS,
+ RDT_WAIT_FOR_LOCAL_FLUSH
+} RetainDeadTuplesPhase;
+
+/*
+ * Critical information for managing phase transitions within the
+ * RetainDeadTuplesPhase.
+ */
+typedef struct RetainDeadTuplesData
+{
+ RetainDeadTuplesPhase phase; /* current phase */
+ XLogRecPtr remote_lsn; /* WAL write position on the publisher */
+
+ /*
+ * Oldest transaction ID that was in the commit phase on the publisher.
+ * Use FullTransactionId to prevent issues with transaction ID wraparound,
+ * where a new remote_oldestxid could falsely appear to originate from the
+ * past and block advancement.
+ */
+ FullTransactionId remote_oldestxid;
+
+ /*
+ * Next transaction ID to be assigned on the publisher. Use
+ * FullTransactionId for consistency and to allow straightforward
+ * comparisons with remote_oldestxid.
+ */
+ FullTransactionId remote_nextxid;
+
+ TimestampTz reply_time; /* when the publisher responds with status */
+
+ /*
+ * Publisher transaction ID that must be awaited to complete before
+ * entering the final phase (RDT_WAIT_FOR_LOCAL_FLUSH). Use
+ * FullTransactionId for the same reason as remote_nextxid.
+ */
+ FullTransactionId remote_wait_for;
+
+ TransactionId candidate_xid; /* candidate for the non-removable
+ * transaction ID */
+ TimestampTz flushpos_update_time; /* when the remote flush position was
+ * updated in final phase
+ * (RDT_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 */
+} RetainDeadTuplesData;
+
+/*
+ * 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 =
{
@@ -332,6 +496,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
+ * and use this information both while sending feedback to the server and
+ * advancing oldest_nonremovable_xid.
+ */
+static XLogRecPtr last_flushpos = InvalidXLogRecPtr;
+
typedef struct SubXactInfo
{
TransactionId xid; /* XID of the subxact */
@@ -372,6 +543,19 @@ static void stream_close_file(void);
static void send_feedback(XLogRecPtr recvpos, bool force, bool requestReply);
+static void maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static bool can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data);
+static void process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void get_candidate_xid(RetainDeadTuplesData *rdt_data);
+static void request_publisher_status(RetainDeadTuplesData *rdt_data);
+static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
+ bool new_xid_found);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -3577,6 +3761,7 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
bool ping_sent = false;
TimeLineID tli;
ErrorContextCallback errcallback;
+ RetainDeadTuplesData rdt_data = {0};
/*
* Init the ApplyMessageContext which we clean up after each replication
@@ -3655,6 +3840,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_recv_timestamp = GetCurrentTimestamp();
ping_sent = false;
+ rdt_data.last_recv_time = last_recv_timestamp;
+
/* Ensure we are reading the data into our memory context. */
MemoryContextSwitchTo(ApplyMessageContext);
@@ -3681,6 +3868,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
UpdateWorkerStats(last_received, send_time, false);
apply_dispatch(&s);
+
+ maybe_advance_nonremovable_xid(&rdt_data, false);
}
else if (c == 'k')
{
@@ -3696,8 +3885,31 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
last_received = end_lsn;
send_feedback(last_received, reply_requested, false);
+
+ maybe_advance_nonremovable_xid(&rdt_data, false);
+
UpdateWorkerStats(last_received, timestamp, true);
}
+ else if (c == 's') /* Primary status update */
+ {
+ rdt_data.remote_lsn = pq_getmsgint64(&s);
+ rdt_data.remote_oldestxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rdt_data.remote_nextxid = FullTransactionIdFromU64((uint64) pq_getmsgint64(&s));
+ rdt_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(rdt_data.remote_lsn))
+ elog(ERROR, "cannot get the latest WAL position from the publisher");
+
+ maybe_advance_nonremovable_xid(&rdt_data, true);
+
+ UpdateWorkerStats(last_received, rdt_data.reply_time, false);
+ }
/* other message types are purposefully ignored */
MemoryContextReset(ApplyMessageContext);
@@ -3710,6 +3922,11 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/* confirm all writes so far */
send_feedback(last_received, false, false);
+ /* Reset the timestamp if no message was received */
+ rdt_data.last_recv_time = 0;
+
+ maybe_advance_nonremovable_xid(&rdt_data, false);
+
if (!in_remote_transaction && !in_streamed_transaction)
{
/*
@@ -3744,6 +3961,14 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
else
wait_time = NAPTIME_PER_CYCLE;
+ /*
+ * Ensure to wake up when it's possible to advance the non-removable
+ * transaction ID.
+ */
+ if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ rdt_data.xid_advance_interval)
+ wait_time = Min(wait_time, rdt_data.xid_advance_interval);
+
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
@@ -3807,6 +4032,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
send_feedback(last_received, requestReply, requestReply);
+ maybe_advance_nonremovable_xid(&rdt_data, false);
+
/*
* Force reporting to ensure long idle periods don't lead to
* arbitrarily delayed stats. Stats can only be reported outside
@@ -3842,7 +4069,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;
@@ -3920,6 +4146,367 @@ send_feedback(XLogRecPtr recvpos, bool force, bool requestReply)
last_flushpos = flushpos;
}
+/*
+ * Attempt to advance the non-removable transaction ID.
+ *
+ * See comments atop worker.c for details.
+ */
+static void
+maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
+ bool status_received)
+{
+ if (!can_advance_nonremovable_xid(rdt_data))
+ return;
+
+ process_rdt_phase_transition(rdt_data, status_received);
+}
+
+/*
+ * Preliminary check to determine if advancing the non-removable transaction ID
+ * is allowed.
+ */
+static bool
+can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * It is sufficient to manage non-removable transaction ID for a
+ * subscription by the main apply worker to detect conflicts reliably even
+ * for table sync or parallel apply workers.
+ */
+ if (!am_leader_apply_worker())
+ return false;
+
+ /* No need to advance if retaining dead tuples is not required */
+ if (!MySubscription->retaindeadtuples)
+ return false;
+
+ return true;
+}
+
+/*
+ * Process phase transitions during the non-removable transaction ID
+ * advancement. See comments atop worker.c for details of the transition.
+ */
+static void
+process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
+ bool status_received)
+{
+ switch (rdt_data->phase)
+ {
+ case RDT_GET_CANDIDATE_XID:
+ get_candidate_xid(rdt_data);
+ break;
+ case RDT_REQUEST_PUBLISHER_STATUS:
+ request_publisher_status(rdt_data);
+ break;
+ case RDT_WAIT_FOR_PUBLISHER_STATUS:
+ wait_for_publisher_status(rdt_data, status_received);
+ break;
+ case RDT_WAIT_FOR_LOCAL_FLUSH:
+ wait_for_local_flush(rdt_data);
+ break;
+ }
+}
+
+/*
+ * Workhorse for the RDT_GET_CANDIDATE_XID phase.
+ */
+static void
+get_candidate_xid(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId oldest_running_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 = rdt_data->last_recv_time ? rdt_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(rdt_data->candidate_xid_time, now,
+ rdt_data->xid_advance_interval))
+ return;
+
+ /*
+ * Immediately update the timer, even if the function returns later
+ * without setting candidate_xid due to inactivity on the subscriber. This
+ * avoids frequent calls to GetOldestActiveTransactionId.
+ */
+ rdt_data->candidate_xid_time = now;
+
+ /*
+ * Consider transactions in the current database, as only dead tuples from
+ * this database are required for conflict detection.
+ */
+ oldest_running_xid = GetOldestActiveTransactionId(false, false);
+
+ /*
+ * Oldest active transaction ID (oldest_running_xid) can't be behind any
+ * of its previously computed value.
+ */
+ Assert(TransactionIdPrecedesOrEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid));
+
+ /* Return if the oldest_nonremovable_xid cannot be advanced */
+ if (TransactionIdEquals(MyLogicalRepWorker->oldest_nonremovable_xid,
+ oldest_running_xid))
+ {
+ adjust_xid_advance_interval(rdt_data, false);
+ return;
+ }
+
+ adjust_xid_advance_interval(rdt_data, true);
+
+ rdt_data->candidate_xid = oldest_running_xid;
+ rdt_data->phase = RDT_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_REQUEST_PUBLISHER_STATUS phase.
+ */
+static void
+request_publisher_status(RetainDeadTuplesData *rdt_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);
+
+ rdt_data->phase = RDT_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 RDT_WAIT_FOR_PUBLISHER_STATUS phase.
+ */
+static void
+wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
+ bool status_received)
+{
+ /*
+ * Return if we have requested but not yet received the publisher status.
+ */
+ if (!status_received)
+ return;
+
+ if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
+ rdt_data->remote_wait_for = rdt_data->remote_nextxid;
+
+ /*
+ * 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.
+ *
+ * It's possible that transactions in the commit phase during the last
+ * cycle have now finished committing, but remote_oldestxid remains older
+ * than remote_wait_for. This can happen if some old transaction came in
+ * the commit phase when we requested status in this cycle. We do not
+ * handle this case explicitly as it's rare and the benefit doesn't
+ * justify the required complexity. Tracking would require either caching
+ * all xids at the publisher or sending them to subscribers. The condition
+ * will resolve naturally once the remaining transactions are finished.
+ *
+ * Directly advancing the non-removable transaction ID is possible if
+ * there are no activities on the publisher since the last advancement
+ * cycle. However, it requires maintaining two fields, last_remote_nextxid
+ * and last_remote_lsn, within the structure for comparison with the
+ * current cycle's values. Considering the minimal cost of continuing in
+ * RDT_WAIT_FOR_LOCAL_FLUSH without awaiting changes, we opted not to
+ * advance the transaction ID here.
+ */
+ if (FullTransactionIdPrecedesOrEquals(rdt_data->remote_wait_for,
+ rdt_data->remote_oldestxid))
+ rdt_data->phase = RDT_WAIT_FOR_LOCAL_FLUSH;
+ else
+ rdt_data->phase = RDT_REQUEST_PUBLISHER_STATUS;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_WAIT_FOR_LOCAL_FLUSH phase.
+ */
+static void
+wait_for_local_flush(RetainDeadTuplesData *rdt_data)
+{
+ Assert(!XLogRecPtrIsInvalid(rdt_data->remote_lsn) &&
+ TransactionIdIsValid(rdt_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 conflicts reliably. This check primarily addresses
+ * scenarios where the publisher's clock falls behind; if the publisher's
+ * clock is ahead, subsequent transactions will naturally bear later
+ * commit timestamps, conforming to the design outlined atop worker.c.
+ *
+ * XXX Consider waiting for the publisher's clock to catch up with the
+ * subscriber's before proceeding to the next phase.
+ */
+ if (TimestampDifferenceExceeds(rdt_data->reply_time,
+ rdt_data->candidate_xid_time, 0))
+ ereport(ERROR,
+ errmsg_internal("oldest_nonremovable_xid transaction ID could be advanced prematurely"),
+ errdetail_internal("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 rdt_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 costly operations in get_flush_position() too frequently
+ * during change application.
+ */
+ if (last_flushpos < rdt_data->remote_lsn && rdt_data->last_recv_time &&
+ TimestampDifferenceExceeds(rdt_data->flushpos_update_time,
+ rdt_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;
+
+ rdt_data->flushpos_update_time = rdt_data->last_recv_time;
+ }
+
+ /* Return to wait for the changes to be applied */
+ if (last_flushpos < rdt_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 = rdt_data->candidate_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ rdt_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. We can even use
+ * flushpos_update_time in the next round to decide whether to get the
+ * latest flush position.
+ */
+ rdt_data->phase = RDT_GET_CANDIDATE_XID;
+ rdt_data->remote_lsn = InvalidXLogRecPtr;
+ rdt_data->remote_oldestxid = InvalidFullTransactionId;
+ rdt_data->remote_nextxid = InvalidFullTransactionId;
+ rdt_data->reply_time = 0;
+ rdt_data->remote_wait_for = InvalidFullTransactionId;
+ rdt_data->candidate_xid = InvalidTransactionId;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_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(RetainDeadTuplesData *rdt_data, bool new_xid_found)
+{
+ if (!new_xid_found && rdt_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.
+ */
+ rdt_data->xid_advance_interval = Min(rdt_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.
+ */
+ rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
+ }
+}
+
/*
* Exit routine for apply workers due to subscription parameter changes.
*/
@@ -4708,6 +5295,30 @@ InitializeLogRepWorker(void)
apply_worker_exit();
}
+ /*
+ * Restart the worker if retain_dead_tuples was enabled during startup.
+ *
+ * At this point, the replication slot used for conflict detection might
+ * not exist yet, or could be dropped soon if the launcher perceives
+ * retain_dead_tuples as disabled. To avoid unnecessary tracking of
+ * oldest_nonremovable_xid when the slot is absent or at risk of being
+ * dropped, a restart is initiated.
+ *
+ * The oldest_nonremovable_xid should be initialized only when the
+ * retain_dead_tuples is enabled before launching the worker. See
+ * logicalrep_worker_launch.
+ */
+ if (am_leader_apply_worker() &&
+ MySubscription->retaindeadtuples &&
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart because the option %s was enabled during startup",
+ MySubscription->name, "retain_dead_tuples"));
+
+ apply_worker_exit();
+ }
+
/* Setup synchronous commit according to the user's wishes */
SetConfigOption("synchronous_commit", MySubscription->synccommit,
PGC_BACKEND, PGC_S_OVERRIDE);
@@ -4864,6 +5475,14 @@ DisableSubscriptionAndExit(void)
errmsg("subscription \"%s\" has been disabled because of an error",
MySubscription->name));
+ /*
+ * Skip the track_commit_timestamp check when disabling the worker due to
+ * an error, as verifying commit timestamps is unnecessary in this
+ * context.
+ */
+ if (MySubscription->retaindeadtuples)
+ CheckSubDeadTupleRetention(false, true, WARNING);
+
proc_exit(0);
}
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index e44ad576bc7..8605776ad86 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 IsSlotForConflictCheck(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,31 @@ ReplicationSlotValidateName(const char *name, int elevel)
return false;
}
}
+
+ if (!allow_reserved_name && IsSlotForConflictCheck(name))
+ {
+ ereport(elevel,
+ errcode(ERRCODE_RESERVED_NAME),
+ errmsg("replication slot name \"%s\" is reserved",
+ name),
+ errdetail("The name \"%s\" is reserved for the conflict detection slot.",
+ CONFLICT_DETECTION_SLOT));
+
+ return false;
+ }
+
return true;
}
+/*
+ * Return true if the replication slot name is "pg_conflict_detection".
+ */
+static bool
+IsSlotForConflictCheck(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 +358,12 @@ ReplicationSlotCreate(const char *name, bool db_specific,
Assert(MyReplicationSlot == NULL);
- ReplicationSlotValidateName(name, ERROR);
+ /*
+ * The logical launcher or pg_upgrade may create or migrate an internal
+ * slot, so using a reserved name is allowed in these cases.
+ */
+ ReplicationSlotValidateName(name, IsBinaryUpgrade || IsLogicalLauncher(),
+ ERROR);
if (failover)
{
@@ -581,6 +614,17 @@ retry:
name)));
}
+ /*
+ * Do not allow users to acquire the reserved slot. This scenario may
+ * occur if the launcher that owns the slot has terminated unexpectedly
+ * due to an error, and a backend process attempts to reuse the slot.
+ */
+ if (!IsLogicalLauncher() && IsSlotForConflictCheck(name))
+ ereport(ERROR,
+ errcode(ERRCODE_UNDEFINED_OBJECT),
+ errmsg("cannot acquire replication slot \"%s\"", name),
+ errdetail("The slot is reserved for conflict detection and can only be acquired by logical replication launcher."));
+
/*
* This is the slot we want; check if it's active under some other
* process. In single user mode, we don't need this check.
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index 28b8591efa5..4c72a0d43b3 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,60 @@ ProcessStandbyHSFeedbackMessage(void)
}
}
+/*
+ * Process the request for a primary status update message.
+ */
+static void
+ProcessStandbyPSRequestMessage(void)
+{
+ XLogRecPtr lsn = InvalidXLogRecPtr;
+ TransactionId oldestXidInCommit;
+ FullTransactionId nextFullXid;
+ FullTransactionId fullOldestXidInCommit;
+ 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);
+
+ /*
+ * Consider transactions in the current database, as only these are the
+ * ones replicated.
+ */
+ oldestXidInCommit = GetOldestActiveTransactionId(true, false);
+ nextFullXid = ReadNextFullTransactionId();
+ fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
+ oldestXidInCommit);
+ lsn = GetXLogWriteRecPtr();
+
+ elog(DEBUG2, "sending primary status");
+
+ /* construct the message... */
+ resetStringInfo(&output_message);
+ pq_sendbyte(&output_message, 's');
+ pq_sendint64(&output_message, lsn);
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(fullOldestXidInCommit));
+ pq_sendint64(&output_message, (int64) U64FromFullTransactionId(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 2418967def6..bf987aed8d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -2814,8 +2814,10 @@ GetRunningTransactionData(void)
*
* Similar to GetSnapshotData but returns just oldestActiveXid. We include
* all PGPROCs with an assigned TransactionId, even VACUUM processes.
- * We look at all databases, though there is no need to include WALSender
- * since this has no effect on hot standby conflicts.
+ *
+ * If allDbs is true, we look at all databases, though there is no need to
+ * include WALSender since this has no effect on hot standby conflicts. If
+ * allDbs is false, skip processes attached to other databases.
*
* This is never executed during recovery so there is no need to look at
* KnownAssignedXids.
@@ -2823,9 +2825,12 @@ GetRunningTransactionData(void)
* We don't worry about updating other counters, we want to keep this as
* simple as possible and leave GetSnapshotData() as the primary code for
* that bookkeeping.
+ *
+ * inCommitOnly indicates getting the oldestActiveXid among the transactions
+ * in the commit critical section.
*/
TransactionId
-GetOldestActiveTransactionId(void)
+GetOldestActiveTransactionId(bool inCommitOnly, bool allDbs)
{
ProcArrayStruct *arrayP = procArray;
TransactionId *other_xids = ProcGlobal->xids;
@@ -2852,6 +2857,8 @@ GetOldestActiveTransactionId(void)
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]);
@@ -2859,6 +2866,13 @@ GetOldestActiveTransactionId(void)
if (!TransactionIdIsNormal(xid))
continue;
+ if (inCommitOnly &&
+ (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
+ if (!allDbs && proc->databaseId != MyDatabaseId)
+ continue;
+
if (TransactionIdPrecedes(xid, oldestRunningXid))
oldestRunningXid = xid;
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index d44f8c262ba..a4f8b4faa90 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -21,6 +21,7 @@
#include "commands/extension.h"
#include "miscadmin.h"
#include "replication/logical.h"
+#include "replication/logicallauncher.h"
#include "replication/origin.h"
#include "replication/worker_internal.h"
#include "storage/lmgr.h"
@@ -410,3 +411,21 @@ binary_upgrade_replorigin_advance(PG_FUNCTION_ARGS)
PG_RETURN_VOID();
}
+
+/*
+ * binary_upgrade_create_conflict_detection_slot
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins.
+ */
+Datum
+binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
+{
+ CHECK_IS_BINARY_UPGRADE;
+
+ CreateConflictDetectionSlot();
+
+ ReplicationSlotRelease();
+
+ PG_RETURN_VOID();
+}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index c6226175528..ca1f3f026f8 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -4962,6 +4962,7 @@ getSubscriptions(Archive *fout)
int i_suboriginremotelsn;
int i_subenabled;
int i_subfailover;
+ int i_subretaindeadtuples;
int i,
ntups;
@@ -5034,10 +5035,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 >= 190000)
+ appendPQExpBufferStr(query,
+ " s.subretaindeadtuples\n");
+ else
+ appendPQExpBufferStr(query,
+ " false AS subretaindeadtuples\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5071,6 +5079,7 @@ getSubscriptions(Archive *fout)
i_subpasswordrequired = PQfnumber(res, "subpasswordrequired");
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
+ i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5104,6 +5113,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].subretaindeadtuples =
+ (strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5362,6 +5373,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subfailover)
appendPQExpBufferStr(query, ", failover = true");
+ if (subinfo->subretaindeadtuples)
+ appendPQExpBufferStr(query, ", retain_dead_tuples = 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 39eef1d6617..41b3f39cee4 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -709,6 +709,7 @@ typedef struct _SubscriptionInfo
bool subpasswordrequired;
bool subrunasowner;
bool subfailover;
+ bool subretaindeadtuples;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/pg_upgrade/check.c b/src/bin/pg_upgrade/check.c
index 30579ef2051..5e6403f0773 100644
--- a/src/bin/pg_upgrade/check.c
+++ b/src/bin/pg_upgrade/check.c
@@ -28,7 +28,7 @@ static void check_for_pg_role_prefix(ClusterInfo *cluster);
static void check_for_new_tablespace_dir(void);
static void check_for_user_defined_encoding_conversions(ClusterInfo *cluster);
static void check_for_unicode_update(ClusterInfo *cluster);
-static void check_new_cluster_logical_replication_slots(void);
+static void check_new_cluster_replication_slots(void);
static void check_new_cluster_subscription_configuration(void);
static void check_old_cluster_for_valid_slots(void);
static void check_old_cluster_subscription_state(void);
@@ -631,7 +631,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();
}
@@ -764,7 +764,7 @@ check_new_cluster(void)
check_for_new_tablespace_dir();
- check_new_cluster_logical_replication_slots();
+ check_new_cluster_replication_slots();
check_new_cluster_subscription_configuration();
}
@@ -2040,48 +2040,80 @@ check_for_unicode_update(ClusterInfo *cluster)
}
/*
- * check_new_cluster_logical_replication_slots()
+ * check_new_cluster_replication_slots()
*
- * Verify that there are no logical replication slots on the new cluster and
- * that the parameter settings necessary for creating slots are sufficient.
+ * Validate the new cluster's readiness for migrating replication slots:
+ * - Ensures no existing logical replication slots on the new cluster when
+ * migrating logical slots.
+ * - Ensure conflict detection slot does not exist on the new cluster when
+ * migrating subscriptions with retain_dead_tuples enabled.
+ * - Ensure that the parameter settings on the new cluster necessary for
+ * creating slots are sufficient.
*/
static void
-check_new_cluster_logical_replication_slots(void)
+check_new_cluster_replication_slots(void)
{
PGresult *res;
PGconn *conn;
int nslots_on_old;
int nslots_on_new;
+ int rdt_slot_on_new;
int max_replication_slots;
char *wal_level;
+ int i_nslots_on_new;
+ int i_rdt_slot_on_new;
- /* Logical slots can be migrated since PG17. */
+ /*
+ * Logical slots can be migrated since PG17 and a physical slot
+ * CONFLICT_DETECTION_SLOT can be migrated since PG19.
+ */
if (GET_MAJOR_VERSION(old_cluster.major_version) <= 1600)
return;
nslots_on_old = count_old_cluster_logical_slots();
- /* Quick return if there are no logical slots to be migrated. */
- if (nslots_on_old == 0)
+ /*
+ * Quick return if there are no slots to be migrated and no subscriptions
+ * have the retain_dead_tuples option enabled.
+ */
+ if (nslots_on_old == 0 && !old_cluster.sub_retain_dead_tuples)
return;
conn = connectToServer(&new_cluster, "template1");
- prep_status("Checking for new cluster logical replication slots");
+ prep_status("Checking for new cluster replication slots");
- res = executeQueryOrDie(conn, "SELECT count(*) "
- "FROM pg_catalog.pg_replication_slots "
- "WHERE slot_type = 'logical' AND "
- "temporary IS FALSE;");
+ res = executeQueryOrDie(conn, "SELECT %s AS nslots_on_new, %s AS rdt_slot_on_new "
+ "FROM pg_catalog.pg_replication_slots",
+ nslots_on_old > 0
+ ? "COUNT(*) FILTER (WHERE slot_type = 'logical' AND temporary IS FALSE)"
+ : "0",
+ old_cluster.sub_retain_dead_tuples
+ ? "COUNT(*) FILTER (WHERE slot_name = 'pg_conflict_detection')"
+ : "0");
if (PQntuples(res) != 1)
- pg_fatal("could not count the number of logical replication slots");
+ pg_fatal("could not count the number of replication slots");
- nslots_on_new = atoi(PQgetvalue(res, 0, 0));
+ i_nslots_on_new = PQfnumber(res, "nslots_on_new");
+ i_rdt_slot_on_new = PQfnumber(res, "rdt_slot_on_new");
+
+ nslots_on_new = atoi(PQgetvalue(res, 0, i_nslots_on_new));
if (nslots_on_new)
+ {
+ Assert(nslots_on_old);
pg_fatal("expected 0 logical replication slots but found %d",
nslots_on_new);
+ }
+
+ rdt_slot_on_new = atoi(PQgetvalue(res, 0, i_rdt_slot_on_new));
+
+ if (rdt_slot_on_new)
+ {
+ Assert(old_cluster.sub_retain_dead_tuples);
+ pg_fatal("The replication slot \"pg_conflict_detection\" already exists on the new cluster");
+ }
PQclear(res);
@@ -2094,12 +2126,24 @@ check_new_cluster_logical_replication_slots(void)
wal_level = PQgetvalue(res, 0, 0);
- if (strcmp(wal_level, "logical") != 0)
+ if (nslots_on_old > 0 && strcmp(wal_level, "logical") != 0)
pg_fatal("\"wal_level\" must be \"logical\" but is set to \"%s\"",
wal_level);
+ if (old_cluster.sub_retain_dead_tuples &&
+ strcmp(wal_level, "minimal") == 0)
+ pg_fatal("\"wal_level\" must be \"replica\" or \"logical\" but is set to \"%s\"",
+ wal_level);
+
max_replication_slots = atoi(PQgetvalue(res, 1, 0));
+ if (old_cluster.sub_retain_dead_tuples &&
+ 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);
+
if (nslots_on_old > max_replication_slots)
pg_fatal("\"max_replication_slots\" (%d) must be greater than or equal to the number of "
"logical replication slots (%d) on the old cluster",
@@ -2211,6 +2255,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 slot since PG19.
+ */
+ 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 4b7a56f5b3b..a437067cdca 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_dead_tuples;
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) >= 1900)
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "COUNT(CASE WHEN subretaindeadtuples THEN 1 END) > 0 AS retain_dead_tuples "
+ "FROM pg_catalog.pg_subscription");
+ else
+ res = executeQueryOrDie(conn, "SELECT count(*) AS nsub,"
+ "'f' AS retain_dead_tuples "
+ "FROM pg_catalog.pg_subscription");
+
+ i_nsub = PQfnumber(res, "nsub");
+ i_retain_dead_tuples = PQfnumber(res, "retain_dead_tuples");
+
+ cluster->nsubs = atoi(PQgetvalue(res, 0, i_nsub));
+ cluster->sub_retain_dead_tuples = (strcmp(PQgetvalue(res, 0, i_retain_dead_tuples), "t") == 0);
PQclear(res);
PQfinish(conn);
diff --git a/src/bin/pg_upgrade/pg_upgrade.c b/src/bin/pg_upgrade/pg_upgrade.c
index 536e49d2616..d5cd5bf0b3a 100644
--- a/src/bin/pg_upgrade/pg_upgrade.c
+++ b/src/bin/pg_upgrade/pg_upgrade.c
@@ -67,6 +67,7 @@ static void set_frozenxids(bool minmxid_only);
static void make_outputdirs(char *pgdata);
static void setup(char *argv0);
static void create_logical_replication_slots(void);
+static void create_conflict_detection_slot(void);
ClusterInfo old_cluster,
new_cluster;
@@ -88,6 +89,7 @@ int
main(int argc, char **argv)
{
char *deletion_script_file_name = NULL;
+ bool migrate_logical_slots;
/*
* pg_upgrade doesn't currently use common/logging.c, but initialize it
@@ -198,18 +200,39 @@ main(int argc, char **argv)
new_cluster.pgdata);
check_ok();
+ migrate_logical_slots = count_old_cluster_logical_slots();
+
/*
- * Migrate the logical slots to the new cluster. Note that we need to do
- * this after resetting WAL because otherwise the required WAL would be
- * removed and slots would become unusable. There is a possibility that
- * background processes might generate some WAL before we could create the
- * slots in the new cluster but we can ignore that WAL as that won't be
- * required downstream.
+ * Migrate replication slots to the new cluster.
+ *
+ * Note that we must migrate logical slots after resetting WAL because
+ * otherwise the required WAL would be removed and slots would become
+ * unusable. There is a possibility that background processes might
+ * generate some WAL before we could create the slots in the new cluster
+ * but we can ignore that WAL as that won't be required downstream.
+ *
+ * The conflict detection slot is not affected by concerns related to WALs
+ * as it only retains the dead tuples. It is created here for consistency.
+ * Note that the new conflict detection slot uses the latest transaction
+ * ID as xmin, so it cannot protect dead tuples that existed before the
+ * upgrade. Additionally, commit timestamps and origin data are not
+ * preserved during the upgrade. So, even after creating the slot, the
+ * upgraded subscriber may be unable to detect conflicts or log relevant
+ * commit timestamps and origins when applying changes from the publisher
+ * occurred before the upgrade especially if those changes were not
+ * replicated. It can only protect tuples that might be deleted after the
+ * new cluster starts.
*/
- if (count_old_cluster_logical_slots())
+ if (migrate_logical_slots || old_cluster.sub_retain_dead_tuples)
{
start_postmaster(&new_cluster, true);
- create_logical_replication_slots();
+
+ if (migrate_logical_slots)
+ create_logical_replication_slots();
+
+ if (old_cluster.sub_retain_dead_tuples)
+ create_conflict_detection_slot();
+
stop_postmaster(false);
}
@@ -1025,3 +1048,24 @@ create_logical_replication_slots(void)
return;
}
+
+/*
+ * create_conflict_detection_slot()
+ *
+ * Create a replication slot to retain information necessary for conflict
+ * detection such as dead tuples, commit timestamps, and origins, for migrated
+ * subscriptions with retain_dead_tuples enabled.
+ */
+static void
+create_conflict_detection_slot(void)
+{
+ PGconn *conn_new_template1;
+
+ prep_status("Creating the replication conflict detection slot");
+
+ conn_new_template1 = connectToServer(&new_cluster, "template1");
+ PQclear(executeQueryOrDie(conn_new_template1, "SELECT pg_catalog.binary_upgrade_create_conflict_detection_slot()"));
+ PQfinish(conn_new_template1);
+
+ check_ok();
+}
diff --git a/src/bin/pg_upgrade/pg_upgrade.h b/src/bin/pg_upgrade/pg_upgrade.h
index 69c965bb7d0..e9401430e69 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_dead_tuples; /* whether a subscription enables
+ * retain_dead_tuples. */
} 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 e46f02c6cc6..77387be0f9d 100644
--- a/src/bin/pg_upgrade/t/004_subscription.pl
+++ b/src/bin/pg_upgrade/t/004_subscription.pl
@@ -22,13 +22,13 @@ $publisher->start;
# Initialize the old subscriber node
my $old_sub = PostgreSQL::Test::Cluster->new('old_sub');
-$old_sub->init;
+$old_sub->init(allows_streaming => 'physical');
$old_sub->start;
my $oldbindir = $old_sub->config_data('--bindir');
# Initialize the new subscriber
my $new_sub = PostgreSQL::Test::Cluster->new('new_sub');
-$new_sub->init;
+$new_sub->init(allows_streaming => 'physical');
my $newbindir = $new_sub->config_data('--bindir');
# In a VPATH build, we'll be started in the source directory, but we want
@@ -89,6 +89,54 @@ $publisher->safe_psql('postgres', "DROP PUBLICATION regress_pub1");
$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_dead_tuples 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_dead_tuples = 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
@@ -200,8 +248,9 @@ $old_sub->safe_psql(
rmtree($new_sub->data_dir . "/pg_upgrade_output.d");
# Verify that the upgrade should be successful with tables in 'ready'/'init'
-# state along with retaining the replication origin's remote lsn, subscription's
-# running status, and failover option.
+# state along with retaining the replication origin's remote lsn,
+# subscription's running status, failover option, and retain_dead_tuples
+# option.
$publisher->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
@@ -211,7 +260,7 @@ $publisher->safe_psql(
$old_sub->safe_psql(
'postgres', qq[
CREATE TABLE tab_upgraded1(id int);
- CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true);
+ CREATE SUBSCRIPTION regress_sub4 CONNECTION '$connstr' PUBLICATION regress_pub4 WITH (failover = true, retain_dead_tuples = true);
]);
# Wait till the table tab_upgraded1 reaches 'ready' state
@@ -270,7 +319,8 @@ $new_sub->append_conf('postgresql.conf',
# Check that pg_upgrade is successful when all tables are in ready or in
# init state (tab_upgraded1 table is in ready state and tab_upgraded2 table is
# in init state) along with retaining the replication origin's remote lsn,
-# subscription's running status, and failover option.
+# subscription's running status, failover option, and retain_dead_tuples
+# option.
# ------------------------------------------------------
command_ok(
[
@@ -293,7 +343,8 @@ ok( !-d $new_sub->data_dir . "/pg_upgrade_output.d",
# ------------------------------------------------------
# Check that the data inserted to the publisher when the new subscriber is down
# will be replicated once it is started. Also check that the old subscription
-# states and relations origins are all preserved.
+# states and relations origins are all preserved, and that the conflict
+# detection slot is created.
# ------------------------------------------------------
$publisher->safe_psql(
'postgres', qq[
@@ -303,15 +354,16 @@ $publisher->safe_psql(
$new_sub->start;
-# The subscription's running status and failover option should be preserved
-# in the upgraded instance. So regress_sub4 should still have subenabled and
-# subfailover set to true, while regress_sub5 should have both set to false.
+# The subscription's running status, failover option, and retain_dead_tuples
+# option should be preserved in the upgraded instance. So regress_sub4 should
+# still have subenabled, subfailover, and subretaindeadtuples set to true,
+# while regress_sub5 should have both set to false.
$result = $new_sub->safe_psql('postgres',
- "SELECT subname, subenabled, subfailover FROM pg_subscription ORDER BY subname"
+ "SELECT subname, subenabled, subfailover, subretaindeadtuples FROM pg_subscription ORDER BY subname"
);
-is( $result, qq(regress_sub4|t|t
-regress_sub5|f|f),
- "check that the subscription's running status and failover are preserved"
+is( $result, qq(regress_sub4|t|t|t
+regress_sub5|f|f|f),
+ "check that the subscription's running status, failover, and retain_dead_tuples are preserved"
);
# Subscription relations should be preserved
@@ -330,6 +382,11 @@ $result = $new_sub->safe_psql('postgres',
);
is($result, qq($remote_lsn), "remote_lsn should have been preserved");
+# The conflict detection slot should be created
+$result = $new_sub->safe_psql('postgres',
+ "SELECT xmin IS NOT NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'");
+is($result, qq(t), "conflict detection slot exists");
+
# Resume the initial sync and wait until all tables of subscription
# 'regress_sub5' are synchronized
$new_sub->append_conf('postgresql.conf',
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index dd25d2fe7b8..7a06af48842 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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)
{
@@ -6814,6 +6814,10 @@ describeSubscriptions(const char *pattern, bool verbose)
appendPQExpBuffer(&buf,
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
+ if (pset.sversion >= 190000)
+ appendPQExpBuffer(&buf,
+ ", subretaindeadtuples AS \"%s\"\n",
+ gettext_noop("Retain dead tuples"));
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 37524364290..dbc586c5bc3 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2319,8 +2319,9 @@ match_previous_words(int pattern_id,
/* ALTER SUBSCRIPTION <name> 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_dead_tuples",
+ "run_as_owner", "slot_name", "streaming",
+ "synchronous_commit", "two_phase");
/* ALTER SUBSCRIPTION <name> SKIP ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SKIP", "("))
COMPLETE_WITH("lsn");
@@ -3774,8 +3775,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_dead_tuples",
+ "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_proc.dat b/src/include/catalog/pg_proc.dat
index 1fc19146f46..3ee8fed7e53 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -11801,6 +11801,10 @@
proname => 'binary_upgrade_replorigin_advance', proisstrict => 'f',
provolatile => 'v', proparallel => 'u', prorettype => 'void',
proargtypes => 'text pg_lsn', prosrc => 'binary_upgrade_replorigin_advance' },
+{ oid => '9159', descr => 'for use by pg_upgrade (conflict detection slot)',
+ proname => 'binary_upgrade_create_conflict_detection_slot', proisstrict => 'f',
+ provolatile => 'v', proparallel => 'u', prorettype => 'void',
+ proargtypes => '', prosrc => 'binary_upgrade_create_conflict_detection_slot' },
# conversion functions
{ oid => '4302',
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 20fc329992d..231ef84ec9a 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 subretaindeadtuples; /* True if dead tuples useful for
+ * conflict detection are 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 retaindeadtuples; /* True if dead tuples useful for conflict
+ * detection are 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/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index c2262e46a7f..9b288ad22a6 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -28,4 +28,9 @@ extern void AlterSubscriptionOwner_oid(Oid subid, Oid newOwnerId);
extern char defGetStreamingMode(DefElem *def);
+extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
+
+extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
+ int elevel_for_sub_disabled);
+
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 82b202f3305..b29453e8e4f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -25,8 +25,11 @@ extern void ApplyLauncherShmemInit(void);
extern void ApplyLauncherForgetWorkerStartTime(Oid subid);
extern void ApplyLauncherWakeupAtCommit(void);
+extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
+extern void CreateConflictDetectionSlot(void);
+
extern bool IsLogicalLauncher(void);
extern pid_t GetLeaderApplyWorkerPid(pid_t pid);
diff --git a/src/include/replication/slot.h b/src/include/replication/slot.h
index 76aeeb92242..9020764d253 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.
*
@@ -292,7 +299,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);
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 30b2775952c..0c7b8440a61 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -86,6 +86,16 @@ typedef struct LogicalRepWorker
/* Indicates whether apply can be performed in parallel. */
bool parallel_apply;
+ /*
+ * The changes made by this and later transactions must be retained to
+ * ensure reliable conflict detection during the apply phase.
+ *
+ * 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.
+ */
+ TransactionId oldest_nonremovable_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
@@ -245,7 +255,8 @@ extern List *logicalrep_workers_find(Oid subid, bool only_running,
extern bool logicalrep_worker_launch(LogicalRepWorkerType wtype,
Oid dbid, Oid subid, const char *subname,
Oid userid, Oid relid,
- dsm_handle subworker_dsm);
+ dsm_handle subworker_dsm,
+ bool retain_dead_tuples);
extern void logicalrep_worker_stop(Oid subid, Oid relid);
extern void logicalrep_pa_worker_stop(ParallelApplyWorkerInfo *winfo);
extern void logicalrep_worker_wakeup(Oid subid, Oid relid);
diff --git a/src/include/storage/proc.h b/src/include/storage/proc.h
index 9f9b3fcfbf1..c6f5ebceefd 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 new
+ * flag to ensure all the transactions that have acquired commit timestamp 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 e4877d88e8f..2f4ae06c279 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -55,7 +55,8 @@ extern RunningTransactions GetRunningTransactionData(void);
extern bool TransactionIdIsInProgress(TransactionId xid);
extern TransactionId GetOldestNonRemovableTransactionId(Relation rel);
extern TransactionId GetOldestTransactionIdConsideredRunning(void);
-extern TransactionId GetOldestActiveTransactionId(void);
+extern TransactionId GetOldestActiveTransactionId(bool inCommitOnly,
+ bool allDbs);
extern TransactionId GetOldestSafeDecodingTransactionId(bool catalogOnly);
extern void GetReplicationHorizons(TransactionId *xmin, TransactionId *catalog_xmin);
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index 529b2241731..a98c97f7616 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00012345
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,34 @@ 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
(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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | 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/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - retain_dead_tuples must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = foo);
+ERROR: retain_dead_tuples requires a Boolean value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = false);
+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 dead tuples | 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/00000000
(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 007c9e70374..f0f714fe747 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -287,6 +287,17 @@ ALTER SUBSCRIPTION regress_testsub SET (disable_on_error = true);
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - retain_dead_tuples must be boolean
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, retain_dead_tuples = 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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index d78a6bac16a..7458d7fba7e 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 @@ $node_publisher->start;
# 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
@@ -146,4 +146,195 @@ $node_subscriber->wait_for_log(
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_dead_tuples = 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)");
+
+# 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 conflict detection slot is created on Node B and the xmin
+# value is valid.
+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");
+
+##################################################
+# Check that the retain_dead_tuples option can be enabled only for disabled
+# subscriptions. Validate the NOTICE message during the subscription DDL, and
+# ensure the conflict detection slot is created upon enabling the
+# retain_dead_tuples option.
+##################################################
+
+# Alter retain_dead_tuples for enabled subscription
+my ($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_dead_tuples = true)");
+ok( $stderr =~
+ /ERROR: cannot set option \"retain_dead_tuples\" for enabled subscription/,
+ "altering retain_dead_tuples is not allowed for enabled subscription");
+
+# Disable the subscription
+$node_A->psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+
+# Enable retain_dead_tuples for disabled subscription
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (retain_dead_tuples = true);");
+ok( $stderr =~
+ /NOTICE: deleted rows to detect conflicts would not be removed until the subscription is enabled/,
+ "altering retain_dead_tuples is allowed for disabled subscription");
+
+# Re-enable the subscription
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Confirm that the conflict detection slot is created on Node A 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");
+
+##################################################
+# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
+# enabled. This warns of the possibility of receiving changes from origins
+# other than the publisher.
+##################################################
+
+($cmdret, $stdout, $stderr) = $node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = any);");
+ok( $stderr =~
+ /WARNING: subscription "tap_sub_a_b" enabled retain_dead_tuples but might not reliably detect conflicts for changes from different origins/,
+ "warn of the possibility of receiving changes from origins other than the publisher");
+
+# Reset the origin to none
+$node_A->psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (origin = none);");
+
+###############################################################################
+# 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);
+
+my $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");
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 3 WHERE a = 1;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+($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 advanced to the latest nextXid. If no
+# transactions are running, the apply worker selects nextXid as the candidate
+# for the non-removable xid. See GetOldestActiveTransactionId().
+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");
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) = $node_A->psql(
+ 'postgres', qq(VACUUM (verbose) public.tab;)
+);
+
+ok( $stderr =~
+ qr/1 removed, 1 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+###############################################################################
+# 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();
diff --git a/src/tools/pgindent/typedefs.list b/src/tools/pgindent/typedefs.list
index ff050e93a50..ff373c3f8cc 100644
--- a/src/tools/pgindent/typedefs.list
+++ b/src/tools/pgindent/typedefs.list
@@ -2565,6 +2565,8 @@ RestrictInfo
Result
ResultRelInfo
ResultState
+RetainDeadTuplesData
+RetainDeadTuplesPhase
ReturnSetInfo
ReturnStmt
ReturningClause
--
2.31.1
On Sun, Jul 20, 2025 at 9:00 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Jul 19, 2025 at 10:32 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Jul 19, 2025 at 3:01 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Fri, Jul 18, 2025 at 5:03 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here are some review comments and questions:
--- + if (inCommitOnly && + (proc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0) + continue; +I've not verified yet but is it possible that we exclude XIDs of
processes who are running on other databases?I can't see how, even the comments atop function says: " We look at
all databases, though there is no need to include WALSender since this
has no effect on hot standby conflicts." which indicate that it
shouldn't exlude XIDs of procs who are running on other databases.I think I misunderstood your question. You were asking if possible, we
should exclude XIDs of processes running on other databases in the
above check as for our purpose, we don't need those.
Right.
If so, I agree
with you, we don't need XIDs of other databases as logical WALSender
will anyway won't process transactions in other databases, so we can
exclude those. The function GetOldestActiveTransactionId() is called
from two places in patch get_candidate_xid() and
ProcessStandbyPSRequestMessage(). We don't need to care for XIDs in
other databases at both places but care for
Commit_Critical_Section_Phase when called from
ProcessStandbyPSRequestMessage(). So, we probably need two parameters
to distinguish those cases.
Why do we need to include all XIDs even in the cases called from
ProcessStandbyPSRequestMessage()? I guess that there is no chance that
the changes happening on other (non-subscribed) databases could
conflict with something on the subscriber.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Mon, Jul 21, 2025 at 11:27 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Sun, Jul 20, 2025 at 9:00 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
If so, I agree
with you, we don't need XIDs of other databases as logical WALSender
will anyway won't process transactions in other databases, so we can
exclude those. The function GetOldestActiveTransactionId() is called
from two places in patch get_candidate_xid() and
ProcessStandbyPSRequestMessage(). We don't need to care for XIDs in
other databases at both places but care for
Commit_Critical_Section_Phase when called from
ProcessStandbyPSRequestMessage(). So, we probably need two parameters
to distinguish those cases.Why do we need to include all XIDs even in the cases called from
ProcessStandbyPSRequestMessage()?
No, we don't need all XIDs even in the case of
ProcessStandbyPSRequestMessage(). That is what I wrote: "The function
GetOldestActiveTransactionId() is called from two places in patch
get_candidate_xid() and ProcessStandbyPSRequestMessage(). We don't
need to care for XIDs in other databases at both places ...". Am I
missing something or you misread it?
I guess that there is no chance that
the changes happening on other (non-subscribed) databases could
conflict with something on the subscriber.
Right.
--
With Regards,
Amit Kapila.
On Mon, Jul 21, 2025 at 8:49 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Jul 21, 2025 at 11:27 PM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Sun, Jul 20, 2025 at 9:00 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
If so, I agree
with you, we don't need XIDs of other databases as logical WALSender
will anyway won't process transactions in other databases, so we can
exclude those. The function GetOldestActiveTransactionId() is called
from two places in patch get_candidate_xid() and
ProcessStandbyPSRequestMessage(). We don't need to care for XIDs in
other databases at both places but care for
Commit_Critical_Section_Phase when called from
ProcessStandbyPSRequestMessage(). So, we probably need two parameters
to distinguish those cases.Why do we need to include all XIDs even in the cases called from
ProcessStandbyPSRequestMessage()?No, we don't need all XIDs even in the case of
ProcessStandbyPSRequestMessage(). That is what I wrote: "The function
GetOldestActiveTransactionId() is called from two places in patch
get_candidate_xid() and ProcessStandbyPSRequestMessage(). We don't
need to care for XIDs in other databases at both places ...". Am I
missing something or you misread it?
Oh I misread it. Sorry for the noise.
I guess that there is no chance that
the changes happening on other (non-subscribed) databases could
conflict with something on the subscriber.Right.
I've reviewed the 0001 patch and it looks good to me. The patch still
has XXX comments at several places. Do we want to keep all of them as
they are (i.e., as something like TODO or FIXME)?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Wed, Jul 23, 2025 at 3:51 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
I've reviewed the 0001 patch and it looks good to me.
Thanks, I have pushed the 0001 patch.
The patch still
has XXX comments at several places. Do we want to keep all of them
Yes, those are primarily the ideas for future optimization and or
special notes for some not so obvious design decisions.
--
With Regards,
Amit Kapila.
On Wednesday, July 23, 2025 12:08 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jul 23, 2025 at 3:51 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:I've reviewed the 0001 patch and it looks good to me.
Thanks, I have pushed the 0001 patch.
Thanks for pushing. I have rebased the remaining patches.
I have reordered the patches to prioritize the detection of update_deleted as
the initial patch. This can give us more time to consider the new GUC, since the
performance-related aspects have been documented.
One pervious patch used to prove the possibility of allowing changing the
retain_dead_tuples for enabled subscription, has not yet been rebased. I will
rebase that once all the main patches are stable.
Best Regards,
Hou zj
Attachments:
v52-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v52-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From c4f254488222c811712d0005365cb116dea3268b Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:41:37 +0800
Subject: [PATCH v52 3/3] 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 | 15 ++++-
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
src/backend/utils/adt/pg_upgrade_support.c | 2 +-
src/include/replication/logicallauncher.h | 2 +-
5 files changed, 63 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0d6616857e7..e616381e44a 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5425,7 +5425,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_dead_tuples</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0e785b1a8c..50d52cbe77f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1252,7 +1252,7 @@ ApplyLauncherMain(Datum main_arg)
* applying remote changes that occurred before the
* subscription was enabled.
*/
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(!sub->enabled);
}
if (!sub->enabled)
@@ -1289,6 +1289,12 @@ ApplyLauncherMain(Datum main_arg)
{
can_advance_xmin = false;
stop_retention = false;
+
+ /*
+ * Re-create the slot if it has been invalidated, as retention
+ * has now resumed.
+ */
+ CreateConflictDetectionSlot(true);
}
/*
@@ -1515,10 +1521,15 @@ invalidate_conflict_slot(void)
* conflict detection, if not yet.
*/
void
-CreateConflictDetectionSlot(void)
+CreateConflictDetectionSlot(bool recreate_if_invalid)
{
TransactionId xmin_horizon;
+ /* Drop the invalidated slot and re-create it if requested */
+ if (recreate_if_invalid && MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index cc0309a6f3c..f5a9b046f05 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -586,6 +586,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4210,10 +4212,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4495,6 +4493,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4502,6 +4519,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4544,9 +4562,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_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_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4576,19 +4593,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index a4f8b4faa90..e20fc44adda 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -423,7 +423,7 @@ binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
{
CHECK_IS_BINARY_UPGRADE;
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(false);
ReplicationSlotRelease();
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 6e3007db5f0..5052c394c8f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -29,7 +29,7 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
-extern void CreateConflictDetectionSlot(void);
+extern void CreateConflictDetectionSlot(bool recreate_if_invalid);
extern bool IsLogicalLauncher(void);
--
2.50.1.windows.1
v52-0001-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v52-0001-Support-the-conflict-detection-for-update_delete.patchDownload
From 1b6ebb471ed0107738a1001753bd78d45d860699 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:55:37 +0800
Subject: [PATCH v52 1/3] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 15 +++
doc/src/sgml/monitoring.sgml | 11 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
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 | 65 ++++++----
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/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 31 ++++-
15 files changed, 288 insertions(+), 45 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 97f547b3cc4..a855a89b388 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretaindeadtuples</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index fcac55aefe6..de41500ebd8 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1775,6 +1775,21 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 823afe1b30b..665cdc7d58c 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index b8cd15f3280..e847e39fa84 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f6eca09ee15..e1470e48bd6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1397,6 +1397,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 f262e7a66f7..aa49c7b4d27 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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 97c4e26b586..24d0b4ada4d 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 b59221c4d06..f966dc72edf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -138,9 +138,9 @@
* Each apply worker that enabled retain_dead_tuples option maintains a
* non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
* prevent dead rows from being removed prematurely when the apply worker still
- * needs them to detect conflicts reliably. This helps to retain the required
- * commit_ts module information, which further helps to detect
- * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * needs them to detect update_deleted conflicts. Additionally, this helps to
+ * retain the required commit_ts module information, which further helps to
+ * detect update_origin_differs and delete_origin_differs conflicts reliably, as
* otherwise, vacuum freeze could remove the required information.
*
* The logical replication launcher manages an internal replication slot named
@@ -185,10 +185,10 @@
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -2912,17 +2912,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retaindeadtuples &&
+ 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. */
@@ -3260,18 +3271,29 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retaindeadtuples &&
+ 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;
}
@@ -4172,8 +4194,8 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4374,10 +4396,11 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop worker.c.
+ * required to detect update_deleted reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop
+ * worker.c.
*
* XXX Consider waiting for the publisher's clock to catch up with the
* subscriber's before proceeding to the next phase.
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 3ee8fed7e53..1a3b17ea112 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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 104b059544d..f8a0aadd95b 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"
@@ -759,7 +760,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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0c7b8440a61..42f4fed38ff 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dce8c672b40..56704426431 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2177,6 +2177,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,
@@ -2185,7 +2186,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 7458d7fba7e..1aa16e3e2b4 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -265,6 +268,8 @@ $node_A->psql('postgres',
###############################################################################
# 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
@@ -278,6 +283,8 @@ is($result, qq(1|1
# 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;");
@@ -289,10 +296,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
--
2.50.1.windows.1
v52-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v52-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From b2c5127ddd5dbbbe9208063b18959b9a8eb0770f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v52 2/3] 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_dead_tuples 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_dead_tuples 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_dead_tuples 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_dead_tuples' 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_dead_tuples 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 | 43 ++++++
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 | 120 +++++++++++++++--
src/backend/replication/logical/worker.c | 126 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 348 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..0d6616857e7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_dead_tuples</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 665cdc7d58c..ed0a1ea3f6f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 4187191ea74..27e55bb0f35 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -3007,6 +3007,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index e1470e48bd6..d4eabeffaa7 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 742d9ba68e9..c0e785b1a8c 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;
@@ -100,9 +103,11 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -465,6 +470,8 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1181,6 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1223,6 +1231,13 @@ ApplyLauncherMain(Datum main_arg)
*/
can_advance_xmin &= sub->enabled;
+ /*
+ * Stop the conflict information retention only if all workers
+ * for subscriptions with retain_dead_tuples enabled have
+ * requested it.
+ */
+ stop_retention &= sub->enabled;
+
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1254,8 +1269,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin,
+ &stop_retention);
/* worker is running already */
continue;
@@ -1265,10 +1281,15 @@ ApplyLauncherMain(Datum main_arg)
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_dead_tuples are
* running, disabling the further computation of the minimum
- * nonremovable xid.
+ * nonremovable xid. Similarly, stop the conflict information
+ * retention only if all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
*/
if (sub->retaindeadtuples)
+ {
can_advance_xmin = false;
+ stop_retention = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1314,13 +1335,16 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain dead tuples. Otherwise, if required,
- * advance the slot's xmin to protect dead tuples required for the
- * conflict detection.
+ * Invalidate the slot if requested by all apply workers. Or, drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain dead tuples. Otherwise, if required, advance
+ * the slot's xmin to protect dead tuples required for the conflict
+ * detection.
*/
if (MyReplicationSlot)
{
+ if (retain_dead_tuples && stop_retention)
+ invalidate_conflict_slot();
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
@@ -1358,11 +1382,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention)
{
TransactionId nonremovable_xid;
+ bool stop_conflict_info_retention;
Assert(worker != NULL);
@@ -1374,8 +1403,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_dead_tuples enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1436,6 +1479,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_dead_tuples enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1516,7 +1590,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;
@@ -1593,6 +1667,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->subretaindeadtuples &&
+ !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 f966dc72edf..cc0309a6f3c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3989,7 +3994,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4204,6 +4210,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4347,6 +4357,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4428,6 +4445,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4473,12 +4511,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4486,9 +4533,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4521,6 +4625,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8605776ad86..6a7d2076385 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"},
};
/*
@@ -1665,6 +1666,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();
}
@@ -1783,6 +1789,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s,
}
}
+ if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION)
+ {
+ if (IsSlotForConflictCheck(NameStr(s->data.name)))
+ return RS_INVAL_CONFLICT_RETENTION_DURATION;
+ }
+
return RS_INVAL_NONE;
}
@@ -2044,6 +2056,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 d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 1a3b17ea112..b084ba5a669 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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 e8fc342d1a9..2f459404c9e 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 42f4fed38ff..d1e755b37cf 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining conflict
+ * information. This is used only when retain_dead_tuples 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 56704426431..33253fb2cc6 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2169,9 +2169,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 1aa16e3e2b4..b0c5f83d917 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -249,6 +253,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -276,7 +284,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Wed, Jul 23, 2025 at 12:53 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Wednesday, July 23, 2025 12:08 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jul 23, 2025 at 3:51 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:I've reviewed the 0001 patch and it looks good to me.
Thanks, I have pushed the 0001 patch.
Thanks for pushing. I have rebased the remaining patches.
I have reordered the patches to prioritize the detection of update_deleted as
the initial patch. This can give us more time to consider the new GUC, since the
performance-related aspects have been documented.
Thank You for patches. Please find a few comments for 0001:
1)
+ if (HeapTupleSatisfiesVacuum(tuple, oldestXmin, buf)
== HEAPTUPLE_RECENTLY_DEAD)
+ dead = true;
Shall we name the variable as 'recently_dead' as 'dead' can even mean
HEAPTUPLE_DEAD.
2)
+ if (MySubscription->retaindeadtuples &&
+ FindMostRecentlyDeletedTupleInfo(localrel, remoteslot,
+
&conflicttuple.xmin,
+
&conflicttuple.origin,
+
&conflicttuple.ts) &&
+ conflicttuple.origin != replorigin_session_origin)
+ type = CT_UPDATE_DELETED;
+ else
+ type = CT_UPDATE_MISSING;
Shall the conflict be detected as update_deleted irrespective of origin?
3)
+ /*
+ * 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).
+ */
a) We can use parentheses for below otherwise sentence becomes
confusing due to multiple 'or' with 'either'
(meaning there is no commit timestamp or origin)
b) we can change last line to: See comments atop worker.c for details
4)
+ <para>
+ 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
+ <xref linkend="guc-track-commit-timestamp"/>
+ and <link
linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated only a <literal>update_missing</literal> conflict will
+ arise
+ </para>
4a)
Can we please make the link as:
<link linkend="guc-track-commit-timestamp"><varname>track_commit_timestamp</varname></link>
This is because the current usage of 'xref linkend' gives it a
slightly bigger font in html file, while all other references of this
GUC is normal font.
4b) We need a comma after truncated:
Note that if a tuple cannot be found due to the table being truncated,
only a update_missing conflict will arise.
5)
monitoring.sgml:
+ <para>
+ Number of times the tuple to be updated was deleted by another origin
+ during the application of changes. See <xref
linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
Here we are using the term 'by another origin', while in the rest of
the doc (see confl_update_origin_differs, confl_delete_origin_differs)
we use the term 'by another source'. Shall we keep it the same?
OTOH, I think using 'origin' is better but the rest of the page is
using source. So perhaps changing source to origin everywhere is
better. Thoughts?
This can be changed if needed once we decide on point 2 above.
thanks
Shveta
On Wed, Jul 23, 2025 at 12:53 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Thanks for pushing. I have rebased the remaining patches.
+ * 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.
IIUC, once the tuple is not found during update, the patch does an
additional scan with SnapshotAny to find the DEAD tuple, so that it
can report update_deleted conflict, if one is found. The reason in the
comments to do sequential scan in such cases sound reasonable but I
was thinking if we can do index scans if the pg_conflict_* slot's xmin
is ahead of the RI (or any usable index that can be used during scan)
index_tuple's xmin? Note, we use a similar check with the indcheckxmin
parameter in pg_index though the purpose of that is different. If this
can happen then still in most cases the index scan will happen.
--
With Regards,
Amit Kapila.
On Thu, Jul 24, 2025 at 9:12 AM shveta malik <shveta.malik@gmail.com> wrote:
2) + if (MySubscription->retaindeadtuples && + FindMostRecentlyDeletedTupleInfo(localrel, remoteslot, + &conflicttuple.xmin, + &conflicttuple.origin, + &conflicttuple.ts) && + conflicttuple.origin != replorigin_session_origin) + type = CT_UPDATE_DELETED; + else + type = CT_UPDATE_MISSING;Shall the conflict be detected as update_deleted irrespective of origin?
On thinking more here, I think that we may have the possibility of
UPDATE after DELETE from the same origin only when a publication
selectively publishes certain operations.
1)
Consider a publication that only publishes UPDATE and DELETE
operations. On the publisher, we may perform operations like DELETE,
INSERT, and UPDATE. On the subscriber, only DELETE and UPDATE events
are received. In this case, should we treat the incoming UPDATE as
update_deleted or update_missing?
2)
Another topology could be:
pub1 --> pub2 --> sub (origin=any)
pub1 --> sub
- pub1 publishing only DELETEs to pub2 and the same are published to
sub.
- pub1 publishing only UPDATEs to sub.
Now, consider that on pub1, an UPDATE occurs first, followed by a
DELETE. But on the sub, the events are received in reverse order:
DELETE arrives before UPDATE. Since both operations originated from
the same source (pub1), how delayed UPDATE's conflict should be
interpreted? Should it be detected as update_deleted or
update_missing? Logically, since the DELETE is the more recent
operation, it should be the final one and UPDATE should be ignored.
But if we detect it as update_missing, we might incorrectly apply the
UPDATE.
Thoughts?
thanks
Shveta
On Fri, Jul 25, 2025 at 12:37 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Jul 24, 2025 at 9:12 AM shveta malik <shveta.malik@gmail.com> wrote:
2) + if (MySubscription->retaindeadtuples && + FindMostRecentlyDeletedTupleInfo(localrel, remoteslot, + &conflicttuple.xmin, + &conflicttuple.origin, + &conflicttuple.ts) && + conflicttuple.origin != replorigin_session_origin) + type = CT_UPDATE_DELETED; + else + type = CT_UPDATE_MISSING;Shall the conflict be detected as update_deleted irrespective of origin?
On thinking more here, I think that we may have the possibility of
UPDATE after DELETE from the same origin only when a publication
selectively publishes certain operations.1)
Consider a publication that only publishes UPDATE and DELETE
operations. On the publisher, we may perform operations like DELETE,
INSERT, and UPDATE. On the subscriber, only DELETE and UPDATE events
are received. In this case, should we treat the incoming UPDATE as
update_deleted or update_missing?
If the user is doing subscription only for certain operations like
Update or Delete, she may not be interested in eventual consistency as
some of the data may not be replicated, so a conflict detection
followed by any resolution may not be helpful.
The other point is that if we report update_delete in such cases, it
won't be reliable, sometimes it can be update_missing as vacuum would
have removed the row, OTOH, if we report update_missing, it will
always be the same conflict, and we can document it.
--
With Regards,
Amit Kapila.
On Friday, July 25, 2025 2:33 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jul 23, 2025 at 12:53 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Thanks for pushing. I have rebased the remaining patches.
+ * 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.IIUC, once the tuple is not found during update, the patch does an additional
scan with SnapshotAny to find the DEAD tuple, so that it can report
update_deleted conflict, if one is found. The reason in the comments to do
sequential scan in such cases sound reasonable but I was thinking if we can
do index scans if the pg_conflict_* slot's xmin is ahead of the RI (or any usable
index that can be used during scan) index_tuple's xmin? Note, we use a similar
check with the indcheckxmin parameter in pg_index though the purpose of
that is different. If this can happen then still in most cases the index scan will
happen.
Right, I think it makes sense to do with the index scan when the index's xmin is
less than the conflict detection xmin, as that can ensure that all the tuples
deleted before the index creation or re-indexing are irrelevant for conflict
detection.
I have implemented in the V53 patch set and improved the test to verify both
index and seq scan for dead tuples.
The V53-0001 also includes Shveta's comments in [1]/messages/by-id/CAJpy0uDiyjDzLU-=NGO7PnXB4OLy4+RxJiAySdw=a+YO62JO2g@mail.gmail.com.
Apart from above issue,
I'd like to clarify why we scan all matching dead tuples in the relation to
find the most recently deleted one in the patch, and I will share an example for
the same.
The main reason is that only the latest deletion information is relevant for
resolving conflicts. If the first tuple retrieved is antiquated while a newer
deleted tuple exists, users may incorrectly resolve the remote change by
applying a last-update-win strategy. Here is an example:
1. In a BI-cluster setup, if both nodes initially contain empty data:
Node A: tbl (empty)
Node B: tbl (empty)
2. Then if user do the following operations on Node A and wait for them to be
replicated to Node B:
INSERT (pk, 1)
DELETE (pk, 1) @9:00
INSERT (pk, 1)
The data on both nodes looks like:
Node A: tbl (pk, 1) - live tuple
(pk, 1) - dead tuple - @9:00
Node B: tbl (pk, 1) - live tuple
(pk, 1) - dead tuple - @9:00
3. If a user do DELETE (pk) on Node B @9:02, and do UDPATE (pk, 1)->(pk, 2) on Node A
@9:01.
When applying the UPDATE on Node B, it cannot find the target tuple, so will
search the dead tuples, but there are two dead tuples:
Node B: tbl (pk, 1) - live tuple
(pk, 1) - dead tuple - @9:00
(pk, 1) - dead tuple - @9:02
If we only fetch the first tuple in the scan, it could be either a) the tuple
deleted @9:00 which is older than the remote UPDATE, or b) the tuple deleted
@9:02, which is newer than the remote UPDATE is @9:01. User may choose to apply
the UPDATE for case a) which can cause data inconsistency between nodes
(using last-update-win strategy).
Ideally, we should give the resolve the new dead tuple @9:02, so the resolver
can choose to ignore the remote UDPATE, keeping the data consistent.
[1]: /messages/by-id/CAJpy0uDiyjDzLU-=NGO7PnXB4OLy4+RxJiAySdw=a+YO62JO2g@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v53-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v53-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 6c2c432cf2b1875adc43c01caa7c7c7f35e4c70f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:41:37 +0800
Subject: [PATCH v53 3/3] 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 | 15 ++++-
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
src/backend/utils/adt/pg_upgrade_support.c | 2 +-
src/include/replication/logicallauncher.h | 2 +-
5 files changed, 63 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0d6616857e7..e616381e44a 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5425,7 +5425,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_dead_tuples</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0e785b1a8c..50d52cbe77f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1252,7 +1252,7 @@ ApplyLauncherMain(Datum main_arg)
* applying remote changes that occurred before the
* subscription was enabled.
*/
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(!sub->enabled);
}
if (!sub->enabled)
@@ -1289,6 +1289,12 @@ ApplyLauncherMain(Datum main_arg)
{
can_advance_xmin = false;
stop_retention = false;
+
+ /*
+ * Re-create the slot if it has been invalidated, as retention
+ * has now resumed.
+ */
+ CreateConflictDetectionSlot(true);
}
/*
@@ -1515,10 +1521,15 @@ invalidate_conflict_slot(void)
* conflict detection, if not yet.
*/
void
-CreateConflictDetectionSlot(void)
+CreateConflictDetectionSlot(bool recreate_if_invalid)
{
TransactionId xmin_horizon;
+ /* Drop the invalidated slot and re-create it if requested */
+ if (recreate_if_invalid && MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 26536eaa345..f0290636af3 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -592,6 +592,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4285,10 +4287,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4570,6 +4568,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4577,6 +4594,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4619,9 +4637,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_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_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4651,19 +4668,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index a4f8b4faa90..e20fc44adda 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -423,7 +423,7 @@ binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
{
CHECK_IS_BINARY_UPGRADE;
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(false);
ReplicationSlotRelease();
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 6e3007db5f0..5052c394c8f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -29,7 +29,7 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
-extern void CreateConflictDetectionSlot(void);
+extern void CreateConflictDetectionSlot(bool recreate_if_invalid);
extern bool IsLogicalLauncher(void);
--
2.50.1.windows.1
v53-0001-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v53-0001-Support-the-conflict-detection-for-update_delete.patchDownload
From 7ac49fc29af24bbd3f53c98576b0100cad356e04 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Thu, 24 Jul 2025 17:03:34 +0800
Subject: [PATCH v53] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 16 ++
doc/src/sgml/monitoring.sgml | 11 +
doc/src/sgml/ref/create_subscription.sgml | 5 +-
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 241 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++
src/backend/replication/logical/relation.c | 1 +
src/backend/replication/logical/worker.c | 145 +++++++++++--
src/backend/utils/adt/pgstatfuncs.c | 18 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 12 +-
src/include/replication/conflict.h | 3 +
src/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 65 +++++-
16 files changed, 512 insertions(+), 45 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 97f547b3cc4..a855a89b388 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretaindeadtuples</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index fcac55aefe6..b75777c6e8d 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1775,6 +1775,22 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ The tuple to be updated was recently deleted by another origin. The update
+ will simply be skipped in this scenario. Note that this conflict can only
+ be detected when
+ <link linkend="guc-track-commit-timestamp"><varname>track_commit_timestamp</varname></link>
+ and <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated, only a <literal>update_missing</literal> conflict will
+ arise. Additionally, if the tuple was deleted by the same origin, an
+ <literal>update_missing</literal> conflict will arise.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 823afe1b30b..3d02400c870 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was recently deleted by another
+ source during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index b8cd15f3280..e847e39fa84 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f6eca09ee15..e1470e48bd6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1397,6 +1397,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 f262e7a66f7..9a95bfb3c52 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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,225 @@ BuildConflictIndexInfo(ResultRelInfo *resultRelInfo, Oid conflictindex)
}
}
+/*
+ * If the tuple is identified as dead and was deleted by a transaction with a
+ * more recent commit timestamp, update the transaction ID, deletion time, and
+ * origin information associated with this tuple.
+ */
+static void
+update_recent_dead_tuple_info(TupleTableSlot *scanslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ bool recently_dead = false;
+ TransactionId xmax;
+ TimestampTz localts;
+ RepOriginId localorigin;
+
+ hslot = (BufferHeapTupleTableSlot *) scanslot;
+
+ 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 worker.c for details).
+ */
+ if (HeapTupleSatisfiesVacuum(tuple, oldestxmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ recently_dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!recently_dead)
+ return;
+
+ xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data);
+ if (!TransactionIdIsValid(xmax))
+ return;
+
+ /* 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;
+ }
+}
+
+/*
+ * 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.
+ *
+ * We scan all matching dead tuples in the relation to find the most recently
+ * deleted one, rather than stopping at the first match. This is because only
+ * the latest deletion information is relevant for resolving conflicts.
+ * Returning solely the first, potentially outdated tuple can lead users to
+ * mistakenly apply remote changes using a last-update-win strategy, even when a
+ * more recent deleted tuple is available. See comments atop worker.c for
+ * details.
+ *
+ * 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, and it
+ * should be used only when the index scans could miss deleted tuples, such as
+ * when an index has been re-indexed or re-created using CONCURRENTLY option
+ * 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
+FindRecentlyDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ TransactionId oldestxmin;
+ 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);
+
+ table_rescan(scan, NULL);
+
+ /* Try to find the tuple */
+ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot))
+ {
+ if (!tuples_equal(scanslot, searchslot, eq, indexbitmap))
+ continue;
+
+ update_recent_dead_tuple_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return *delete_time != 0;
+}
+
+/*
+ * Similar to FindRecentlyDeletedTupleInfoSeq() but using index scan to locate
+ * the deleted tuple.
+ */
+bool
+FindRecentlyDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ Relation idxrel;
+ ScanKeyData skey[INDEX_MAX_KEYS];
+ int skey_attoff;
+ IndexScanDesc scan;
+ TupleTableSlot *scanslot;
+ TypeCacheEntry **eq;
+ TransactionId oldestxmin;
+ bool isIdxSafeToSkipDuplicates;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ Assert(OidIsValid(idxoid));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ isIdxSafeToSkipDuplicates = (GetRelationIdentityOrPK(rel) == idxoid);
+ oldestxmin = GetOldestNonRemovableTransactionId(rel);
+
+ scanslot = table_slot_create(rel, NULL);
+
+ idxrel = index_open(idxoid, RowExclusiveLock);
+
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /* Start an index scan. */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, 0);
+
+ index_rescan(scan, skey, skey_attoff, NULL, 0);
+
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ /*
+ * Avoid expensive equality check if the index is primary key or
+ * replica identity index.
+ */
+ if (!isIdxSafeToSkipDuplicates)
+ {
+ if (eq == NULL)
+ eq = palloc0(sizeof(*eq) * scanslot->tts_tupleDescriptor->natts);
+
+ if (!tuples_equal(scanslot, searchslot, eq, NULL))
+ continue;
+ }
+
+ update_recent_dead_tuple_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ index_endscan(scan);
+
+ index_close(idxrel, NoLock);
+
+ 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 97c4e26b586..24d0b4ada4d 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/relation.c b/src/backend/replication/logical/relation.c
index f59046ad620..c93d34923c4 100644
--- a/src/backend/replication/logical/relation.c
+++ b/src/backend/replication/logical/relation.c
@@ -25,6 +25,7 @@
#include "executor/executor.h"
#include "nodes/makefuncs.h"
#include "replication/logicalrelation.h"
+#include "replication/slot.h"
#include "replication/worker_internal.h"
#include "utils/inval.h"
#include "utils/lsyscache.h"
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b59221c4d06..695fad8f761 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -138,9 +138,9 @@
* Each apply worker that enabled retain_dead_tuples option maintains a
* non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
* prevent dead rows from being removed prematurely when the apply worker still
- * needs them to detect conflicts reliably. This helps to retain the required
- * commit_ts module information, which further helps to detect
- * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * needs them to detect update_deleted conflicts. Additionally, this helps to
+ * retain the required commit_ts module information, which further helps to
+ * detect update_origin_differs and delete_origin_differs conflicts reliably, as
* otherwise, vacuum freeze could remove the required information.
*
* The logical replication launcher manages an internal replication slot named
@@ -185,10 +185,10 @@
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -576,6 +576,12 @@ static bool FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel
Oid localidxoid,
TupleTableSlot *remoteslot,
TupleTableSlot **localslot);
+static bool FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid,
+ TupleTableSlot *remoteslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
static void apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *remoteslot,
LogicalRepTupleData *newtup,
@@ -2912,17 +2918,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retaindeadtuples &&
+ FindDeletedTupleInLocalRel(localrel, localindexoid, 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. */
@@ -3142,6 +3159,78 @@ FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel,
return found;
}
+/*
+ * Check whether the index can reliably locate the deleted tuple in the local
+ * relation.
+ *
+ * Considering an index might omit all deleted tuples if it has been re-indexed
+ * or re-created using CONCURRENTLY option during change applications, an index
+ * is deemed usable only when the conflict detection slot.xmin exceeds the index
+ * tuple's xmin in pg_index. This ensures that any tuples deleted before the
+ * index creation or re-indexing are irrelevant for conflict detection.
+ *
+ * Note that this might also abandons some indexes that are updated due to other
+ * operations or without the CONCURRENTLY option. However, this is generally
+ * acceptable, as both the DDL commands that modify indexes and the need to scan
+ * dead tuples for the update_deleted are relatively rare occurrences.
+ */
+static bool
+IsIndexUsableForFindingDeletedTuple(Oid localindexoid)
+{
+ HeapTuple index_tuple;
+ TransactionId index_xmin;
+ TransactionId conflict_detection_xmin;
+ ReplicationSlot *slot;
+
+ index_tuple = SearchSysCache1(INDEXRELID, ObjectIdGetDatum(localindexoid));
+
+ /*
+ * A frozen transaction ID indicates that it must fall behind the conflict
+ * detection slot.xmin.
+ */
+ if (HeapTupleHeaderXminFrozen(index_tuple->t_data))
+ return true;
+
+ slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+
+ Assert(slot);
+
+ SpinLockAcquire(&slot->mutex);
+ conflict_detection_xmin = slot->data.xmin;
+ SpinLockRelease(&slot->mutex);
+
+ index_xmin = HeapTupleHeaderGetRawXmin(index_tuple->t_data);
+
+ return TransactionIdPrecedes(index_xmin, conflict_detection_xmin);
+}
+
+/*
+ * Try to find a deleted tuple in the local relation that matching the values of
+ * the tuple received from the publication side (in 'remoteslot'). The function
+ * uses either replica identity index, primary key, index or if needed,
+ * sequential scan.
+ *
+ * Return true if found the deleted tuple. The transaction ID, commit timestamp,
+ * and origin of the transaction for the deletion, if found, are
+ * stored in '*delete_xid', '*delete_origin', and '*delete_time' respectively.
+ */
+static bool
+FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid, TupleTableSlot *remoteslot,
+ TransactionId *delete_xid, RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ if (OidIsValid(localidxoid) &&
+ IsIndexUsableForFindingDeletedTuple(localidxoid))
+ return FindRecentlyDeletedTupleInfoByIndex(localrel, localidxoid,
+ remoteslot, delete_xid,
+ delete_origin, delete_time);
+ else
+ return FindRecentlyDeletedTupleInfoSeq(localrel, remoteslot,
+ delete_xid, delete_origin,
+ delete_time);
+}
+
/*
* This handles insert, update, delete on a partitioned table.
*/
@@ -3260,18 +3349,31 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retaindeadtuples &&
+ FindDeletedTupleInLocalRel(partrel,
+ part_entry->localindexoid,
+ 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;
}
@@ -4172,8 +4274,8 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4374,10 +4476,11 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop worker.c.
+ * required to detect update_deleted reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop
+ * worker.c.
*
* XXX Consider waiting for the publisher's clock to catch up with the
* subscriber's before proceeding to the next phase.
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 3ee8fed7e53..1a3b17ea112 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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 104b059544d..6b1f89c1feb 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"
@@ -759,7 +760,16 @@ extern bool RelationFindReplTupleByIndex(Relation rel, Oid idxoid,
TupleTableSlot *outslot);
extern bool RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode,
TupleTableSlot *searchslot, TupleTableSlot *outslot);
-
+extern bool FindRecentlyDeletedTupleInfoSeq(Relation rel,
+ TupleTableSlot *searchslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
+extern bool FindRecentlyDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ 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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0c7b8440a61..42f4fed38ff 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dce8c672b40..56704426431 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2177,6 +2177,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,
@@ -2185,7 +2186,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 976d53a870e..0a4968f359d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -270,6 +273,8 @@ $node_A->psql('postgres',
###############################################################################
# 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
@@ -288,6 +293,8 @@ $node_A->poll_query_until('postgres',
"SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
);
+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;");
@@ -299,10 +306,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
@@ -324,6 +351,40 @@ ok( $stderr =~
qr/1 removed, 1 remain, 0 are dead but not yet removable/,
'the deleted column is removed');
+###############################################################################
+# Check that dead tuples can be found through a full table sequential scan
+###############################################################################
+
+# Drop the primary key from tab on node A and set REPLICA IDENTITY to FULL to
+# enforce sequential scanning of the table.
+$node_A->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_B->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_A->safe_psql('postgres', "ALTER TABLE tab DROP CONSTRAINT tab_pkey;");
+
+# Disable the logical replication from node B to node A
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE");
+
+# Wait for the apply worker to stop
+$node_A->poll_query_until('postgres',
+ "SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
+);
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 4 WHERE a = 2;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 2;");
+
+$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 \(2, 4\); replica identity full \(2, 2\)/,
+ 'update target row was deleted in tab');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.50.1.windows.1
v53-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v53-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From ffa27b8b2b283c189429b8f1f31859b72166d354 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v53 2/3] 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_dead_tuples 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_dead_tuples 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_dead_tuples 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_dead_tuples' 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_dead_tuples 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 | 43 ++++++
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 | 120 +++++++++++++++--
src/backend/replication/logical/worker.c | 126 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 348 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..0d6616857e7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_dead_tuples</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3d02400c870..3d56a07ad96 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 4187191ea74..27e55bb0f35 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -3007,6 +3007,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index e1470e48bd6..d4eabeffaa7 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 742d9ba68e9..c0e785b1a8c 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;
@@ -100,9 +103,11 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -465,6 +470,8 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1181,6 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1223,6 +1231,13 @@ ApplyLauncherMain(Datum main_arg)
*/
can_advance_xmin &= sub->enabled;
+ /*
+ * Stop the conflict information retention only if all workers
+ * for subscriptions with retain_dead_tuples enabled have
+ * requested it.
+ */
+ stop_retention &= sub->enabled;
+
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1254,8 +1269,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin,
+ &stop_retention);
/* worker is running already */
continue;
@@ -1265,10 +1281,15 @@ ApplyLauncherMain(Datum main_arg)
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_dead_tuples are
* running, disabling the further computation of the minimum
- * nonremovable xid.
+ * nonremovable xid. Similarly, stop the conflict information
+ * retention only if all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
*/
if (sub->retaindeadtuples)
+ {
can_advance_xmin = false;
+ stop_retention = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1314,13 +1335,16 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain dead tuples. Otherwise, if required,
- * advance the slot's xmin to protect dead tuples required for the
- * conflict detection.
+ * Invalidate the slot if requested by all apply workers. Or, drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain dead tuples. Otherwise, if required, advance
+ * the slot's xmin to protect dead tuples required for the conflict
+ * detection.
*/
if (MyReplicationSlot)
{
+ if (retain_dead_tuples && stop_retention)
+ invalidate_conflict_slot();
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
@@ -1358,11 +1382,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention)
{
TransactionId nonremovable_xid;
+ bool stop_conflict_info_retention;
Assert(worker != NULL);
@@ -1374,8 +1403,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_dead_tuples enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1436,6 +1479,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_dead_tuples enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1516,7 +1590,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;
@@ -1593,6 +1667,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->subretaindeadtuples &&
+ !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 1b3e5f51642..26536eaa345 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4064,7 +4069,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4279,6 +4285,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4422,6 +4432,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4503,6 +4520,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4548,12 +4586,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4561,9 +4608,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4596,6 +4700,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8605776ad86..6a7d2076385 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"},
};
/*
@@ -1665,6 +1666,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();
}
@@ -1783,6 +1789,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s,
}
}
+ if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION)
+ {
+ if (IsSlotForConflictCheck(NameStr(s->data.name)))
+ return RS_INVAL_CONFLICT_RETENTION_DURATION;
+ }
+
return RS_INVAL_NONE;
}
@@ -2044,6 +2056,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 d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 1a3b17ea112..b084ba5a669 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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 e8fc342d1a9..2f459404c9e 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 42f4fed38ff..d1e755b37cf 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining conflict
+ * information. This is used only when retain_dead_tuples 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 56704426431..33253fb2cc6 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2169,9 +2169,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 0a4968f359d..ba323b5109e 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Thursday, July 24, 2025 11:42 AM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Jul 23, 2025 at 12:53 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Wednesday, July 23, 2025 12:08 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:
On Wed, Jul 23, 2025 at 3:51 AM Masahiko Sawada
<sawada.mshk@gmail.com> wrote:I've reviewed the 0001 patch and it looks good to me.
Thanks, I have pushed the 0001 patch.
Thanks for pushing. I have rebased the remaining patches.
Thanks for the comments!
I have reordered the patches to prioritize the detection of
update_deleted as the initial patch. This can give us more time to
consider the new GUC, since the performance-related aspects have beendocumented.
2) + if (MySubscription->retaindeadtuples && + FindMostRecentlyDeletedTupleInfo(localrel, + remoteslot, + &conflicttuple.xmin, + &conflicttuple.origin, + &conflicttuple.ts) && + conflicttuple.origin != replorigin_session_origin) + type = CT_UPDATE_DELETED; + else + type = CT_UPDATE_MISSING;Shall the conflict be detected as update_deleted irrespective of origin?
According to the discussion[1]/messages/by-id/CAA4eK1L09u_A0HFRydA4xc=HpPkCh+7h-+_WRhKw1Cksp5_5zQ@mail.gmail.com, I kept the current behavior.
5) monitoring.sgml: + <para> + Number of times the tuple to be updated was deleted by another origin + during the application of changes. See <xref linkend="conflict-update-deleted"/> + for details about this conflict. + </para></entry>Here we are using the term 'by another origin', while in the rest of the doc (see
confl_update_origin_differs, confl_delete_origin_differs) we use the term 'by
another source'. Shall we keep it the same?
OTOH, I think using 'origin' is better but the rest of the page is using source.
So perhaps changing source to origin everywhere is better. Thoughts?
This can be changed if needed once we decide on point 2 above.
Yes, origin may be better. But for now, I have changed to 'source' here to be
consistent with the descriptions around it, and we can improve it in a separate
patch if needed.
Other comments have been addressed in the V53 patch set.
[1]: /messages/by-id/CAA4eK1L09u_A0HFRydA4xc=HpPkCh+7h-+_WRhKw1Cksp5_5zQ@mail.gmail.com
Best Regards,
Hou zj
Hi All,
We conducted performance testing of a bi-directional logical
replication setup, focusing on the primary use case of the
update_deleted feature.
To simulate a realistic scenario, we used a high workload with limited
concurrent updates, and well-distributed writes among servers.
Used source
===========
pgHead commit 62a17a92833 + v47 patch set
Machine details
===============
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
Test-1: Distributed Write Load
==============================
Highlight:
-----------
- In a bi-directional logical replication setup, with
well-distributed write workloads and a thoughtfully tuned
configuration to minimize lag (e.g., through row filters), TPS
regression is minimal or even negligible.
- Performance can be sustained with significantly fewer apply workers
compared to the number of client connections on the publisher.
Setup:
--------
- 2 Nodes(node1 and node2) are created(on same machine) of same
configurations -
autovacuum = false
shared_buffers = '30GB'
-- Also, worker and logical replication related parameters were
increased as per requirement (see attached scripts for details).
- Both nodes have two set of pgbench tables initiated with *scale=300*:
-- set1: pgbench_pub_accounts, pgbench_pub_tellers,
pgbench_pub_branches, and pgbench_pub_history
-- set2: pgbench_accounts, pgbench_tellers, pgbench_branches, and
pgbench_history
- Node1 is publishing all changes for set1 tables and Node2 has
subscribed for the same.
- Node2 is publishing all changes for set2 tables and Node2 has
subscribed for the same.
Note: In all the tests, subscriptions are created with (origin=NONE)
as it is a bi-directional replication.
Workload Run:
---------------
- On node1, pgbench(read-write) with option "-b simple-update" is run
on set1 tables.
- On node2, pgbench(read-write) with option "-b simple-update" is run
on set2 tables.
- #clients = 40
- pgbench run duration = 10 minutes.
- results were measured for 3 runs of each case.
Test Runs:
- Six tests were done with varying #pub-sub pairs and below is TPS
reduction in both nodes for all the cases:
| Case | # Pub-Sub Pairs | TPS Reduction |
| ---- | --------------- | -------------- |
| 01 | 30 | 0–1% |
| 02 | 15 | 6–7% |
| 03 | 5 | 7–8% |
| 04 | 3 | 0-1% |
| 05 | 2 | 14–15% |
| 06 | 1 (no filters) | 37–40% |
- With appropriate row filters and distribution of load across apply
workers, the performance impact of update_deleted patch can be
minimized.
- Just 3 pub-sub pairs are enough to keep TPS close to the baseline
for the given workload.
- Poor distribution of replication workload (e.g., only 1–2 pub-sub
pairs) leads to higher overhead due to increased apply worker
contention.
~~~~
Detailed results for all the above cases:
case-01:
---------
- Created 30 pub-sub pairs to distribute the replication load between
30 apply workers on each node.
Results:
#run pgHead_Node1_TPS patched_Node1_TPS pgHead_Node2_TPS
patched_Node2_TPS
1 5633.377165 5579.244492 6385.839585 6482.775975
2 5926.328644 5947.035275 6216.045707 6416.113723
3 5522.804663 5542.380108 6541.031535 6190.123097
median 5633.377165 5579.244492 6385.839585 6416.113723
regression -1% 0%
- No regression
~~~~
case-02:
---------
- #pub-sub pairs = 15
Results:
#run pgHead_Node1_TPS patched_Node1_TPS pgHead_Node2_TPS
patched_Node2_TPS
1 8207.708475 7584.288026 8854.017934 8204.301497
2 8120.979334 7404.735801 8719.451895 8169.697482
3 7877.859139 7536.762733 8542.896669 8177.853563
median 8120.979334 7536.762733 8719.451895 8177.853563
regression -7% -6%
- There was 6-7% TPS reduction on both nodes, which seems in acceptable range.
~~~
case-03:
---------
- #pub-sub pairs = 5
Results:
#run pgHead_Node1_TPS patched_Node1_TPS pgHead_Node2_TPS
patched_Node2_TPS
1 12325.90315 11664.7445 12997.47104 12324.025
2 12060.38753 11370.52775 12728.41287 12127.61208
3 12390.3677 11367.10255 13135.02558 12036.71502
median 12325.90315 11370.52775 12997.47104 12127.61208
regression -8% -7%
- There was 7-8% TPS reduction on both nodes, which seems in acceptable range.
~~~
case-04:
---------
- #pub-sub pairs = 3
Results:
#run pgHead_Node1_TPS patched_Node1_TPS pgHead_Node2_TPS
patched_Node2_TPS
1 13186.22898 12464.42604 13973.8394 13370.45596
2 13038.15817 13014.03906 13866.51966 13866.47395
3 13881.10513 13868.71971 14687.67444 14516.33854
median 13186.22898 13014.03906 13973.8394 13866.47395
regression -1% -1%
- No regression observed
case-05:
---------
- #pub-sub pairs = 2
Results:
#run pgHead_Node1_TPS patched_Node1_TPS pgHead_Node2_TPS
patched_Node2_TPS
1 15936.98792 13563.98476 16734.35292 14527.22942
2 16031.23003 13648.24979 16958.49609 14657.80008
3 16113.79935 13550.68329 17029.5035 14509.84068
median 16031.23003 13563.98476 16958.49609 14527.22942
regression -15% -14%
- The TPS reduced by 14-15% on both nodes.
~~~
case-06:
---------
- #pub-sub pairs = 1 , no row filter is used on both nodes
Results:
#run pgHead_Node1_TPS patched_Node1_TPS pgHead_Node2_TPS
patched_Node2_TPS
1 22900.06507 13609.60639 23254.25113 14592.25271
2 22110.98426 13907.62583 22755.89945 14805.73717
3 22719.88901 13246.41484 23055.70406 14256.54223
median 22719.88901 13609.60639 23055.70406 14592.25271
regression -40% -37%
- The regression observed is 37-40% on both nodes.
~~~~
Test-2: High concurrency
===========================
Highlight:
------------
Despite poor write distribution across servers and high concurrent
updates, distributing replication load across multiple apply workers
limited the TPS drop to just 15–18%.
Setup:
---------------
- 2 Nodes(node1 and node2) are created with same configuration as in Test-01
- Both nodes have same set of pgbench tables initialized with
scale=60 (small tables to increase concurrent updates)
- Both nodes are subscribed to each other for all the changes.
-- 15 pub-sub pairs are created using row filters to distribute the
load and all the subscriptions are created with (origin = NONE).
Workload Run:
---------------
- On both nodes,the default pgbench(read-write) is run on tables.
- #clients = 15
- pgbench run duration = 5 minutes.
- results were measured for 2 runs of each case.
Results:
Node1 TPS:
#run pgHead_Node1_TPS patched_Node1_TPS
1 9585.470749 7660.645249
2 9442.364918 8035.531482
median 9513.917834 7848.088366
regression -18%
Node2 TPS:
#run pgHead_Node2_TPS patched_Node2_TPS
1 9485.232611 8248.783417
2 9468.894086 7938.991136
median 9477.063349 8093.887277
regression -15%
- Under high concurrent writes to the same small tables, contention
increases and the TPS drop is 15-18% on both nodes.
~~~~
The scripts used for above tests are attached.
--
Thanks,
Nisha
Attachments:
On Fri, Jul 25, 2025 at 2:31 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jul 25, 2025 at 12:37 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Jul 24, 2025 at 9:12 AM shveta malik <shveta.malik@gmail.com> wrote:
2) + if (MySubscription->retaindeadtuples && + FindMostRecentlyDeletedTupleInfo(localrel, remoteslot, + &conflicttuple.xmin, + &conflicttuple.origin, + &conflicttuple.ts) && + conflicttuple.origin != replorigin_session_origin) + type = CT_UPDATE_DELETED; + else + type = CT_UPDATE_MISSING;Shall the conflict be detected as update_deleted irrespective of origin?
On thinking more here, I think that we may have the possibility of
UPDATE after DELETE from the same origin only when a publication
selectively publishes certain operations.1)
Consider a publication that only publishes UPDATE and DELETE
operations. On the publisher, we may perform operations like DELETE,
INSERT, and UPDATE. On the subscriber, only DELETE and UPDATE events
are received. In this case, should we treat the incoming UPDATE as
update_deleted or update_missing?If the user is doing subscription only for certain operations like
Update or Delete, she may not be interested in eventual consistency as
some of the data may not be replicated, so a conflict detection
followed by any resolution may not be helpful.The other point is that if we report update_delete in such cases, it
won't be reliable, sometimes it can be update_missing as vacuum would
have removed the row, OTOH, if we report update_missing, it will
always be the same conflict, and we can document it.
Agree with both the points. We can keep the current behaviour as it is.
thanks
Shveta
On Fri, Jul 25, 2025 at 4:38 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Right, I think it makes sense to do with the index scan when the index's xmin is
less than the conflict detection xmin, as that can ensure that all the tuples
deleted before the index creation or re-indexing are irrelevant for conflict
detection.I have implemented in the V53 patch set and improved the test to verify both
index and seq scan for dead tuples.
Thanks. Following are a few comments on 0001 patch:
1.
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1397,6 +1397,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.apply_error_count,
ss.sync_error_count,
ss.confl_insert_exists,
+ ss.confl_update_deleted,
…
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
Can we consider splitting stats into a separate patch? It will help us
to first focus on core functionality of detecting update_delete
conflict.
2.
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.
Here we should add at end "and no usable index is found"
3.
+ * We scan all matching dead tuples in the relation to find the most recently
+ * deleted one, rather than stopping at the first match. This is because only
+ * the latest deletion information is relevant for resolving conflicts.
+ * Returning solely the first, potentially outdated tuple can lead users to
+ * mistakenly apply remote changes using a last-update-win strategy,
even when a
+ * more recent deleted tuple is available. See comments atop worker.c for
+ * details.
I think we can share a short example of cases when this can happen.
And probably a test which will fail if the user only fetches the first
dead tuple?
4.
executor\execReplication.c(671) : warning C4700: uninitialized local
variable 'eq' used
Please fix this warning.
5.
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /* Start an index scan. */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, 0);
While scanning with SnapshotAny, isn't it possible that we find some
tuple for which the xact is still not committed or are inserted
successfully just before the scan is started?
I think such tuples shouldn't be considered for giving update_deleted.
It seems the patch will handle it later during
update_recent_dead_tuple_info() where it uses following check: "if
(HeapTupleSatisfiesVacuum(tuple, oldestxmin, buf) ==
HEAPTUPLE_RECENTLY_DEAD)", is my understanding correct? If so, we
should add some comments for it.
6.
FindRecentlyDeletedTupleInfoSeq()
{
…
+ /* 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);
…
...
+ if (!tuples_equal(scanslot, searchslot, eq, indexbitmap))
+ continue;
We don't do any such thing in RelationFindReplTupleSeq(), so, if we do
something differently here, it should be explained in the comments.
--
With Regards,
Amit Kapila.
On Fri, Jul 25, 2025 at 4:38 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
The V53-0001 also includes Shveta's comments in [1].
Thanks, I have not yet completed the review, but please find a few
comments on 001:
1)
IsIndexUsableForFindingDeletedTuple()
We first have:
+ /*
+ * A frozen transaction ID indicates that it must fall behind the conflict
+ * detection slot.xmin.
+ */
+ if (HeapTupleHeaderXminFrozen(index_tuple->t_data))
+ return true;
thent his:
+ index_xmin = HeapTupleHeaderGetRawXmin(index_tuple->t_data);
Shall we use HeapTupleHeaderGetXmin() instead of above 2? We can check
if xid returned by HeapTupleHeaderGetXmin() is FrozenTransactionId or
normal one and then do further processing.
2)
Both FindRecentlyDeletedTupleInfoByIndex and
FindRecentlyDeletedTupleInfoSeq() has:
+ /* Exit early if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
We shall either move this check to FindDeletedTupleInLocalRel() or in
the caller of FindDeletedTupleInLocalRel() where we check
'MySubscription->retaindeadtuples'. Moving to the caller of
FindDeletedTupleInLocalRel() looks better as there is no need to call
FindDeletedTupleInLocalRel itself if pre-conditions are not met.
3)
FindRecentlyDeletedTupleInfoSeq():
+ * 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.
+ */
Shall we extend the last line to:
where the target row for an update cannot be found and index scan can
not be used.
4)
catalogs.sgml:
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
he --> the
thanks
Shveta
On Mon, Jul 28, 2025 at 4:38 PM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, Jul 25, 2025 at 4:38 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:The V53-0001 also includes Shveta's comments in [1].
Thanks, I have not yet completed the review, but please find a few
comments on 001:1) IsIndexUsableForFindingDeletedTuple() We first have: + /* + * A frozen transaction ID indicates that it must fall behind the conflict + * detection slot.xmin. + */ + if (HeapTupleHeaderXminFrozen(index_tuple->t_data)) + return true;thent his:
+ index_xmin = HeapTupleHeaderGetRawXmin(index_tuple->t_data);Shall we use HeapTupleHeaderGetXmin() instead of above 2? We can check
if xid returned by HeapTupleHeaderGetXmin() is FrozenTransactionId or
normal one and then do further processing.2) Both FindRecentlyDeletedTupleInfoByIndex and FindRecentlyDeletedTupleInfoSeq() has: + /* Exit early if the commit timestamp data is not available */ + if (!track_commit_timestamp) + return false;We shall either move this check to FindDeletedTupleInLocalRel() or in
the caller of FindDeletedTupleInLocalRel() where we check
'MySubscription->retaindeadtuples'. Moving to the caller of
FindDeletedTupleInLocalRel() looks better as there is no need to call
FindDeletedTupleInLocalRel itself if pre-conditions are not met.3) FindRecentlyDeletedTupleInfoSeq(): + * 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. + */ Shall we extend the last line to: where the target row for an update cannot be found and index scan can not be used.4)
catalogs.sgml:
+ If true, he detection of <xref linkend="conflict-update-deleted"/> is
he --> the
5)
FindRecentlyDeletedTupleInfoSeq():
+ /* Get the cutoff xmin for HeapTupleSatisfiesVacuum */
+ oldestxmin = GetOldestNonRemovableTransactionId(rel);
Another point is which xid should be used as threshold in
HeapTupleSatisfiesVacuum() to decide if tuple is DEAD or RECENTLY-DEAD
for update_deleted case? Currently we are using
GetOldestNonRemovableTransactionId() but the xid returned here could
be older than pg_conflict_detection's xmin in presence of other
logical slots which have older effective_xmin. Shall we use
pg_conflict_detection's xmin instead as threshold or worker's
oldest_nonremovable_xid?
thanks
Shveta
On Monday, July 28, 2025 7:43 PM shveta malik <shveta.malik@gmail.com> wrote:
On Mon, Jul 28, 2025 at 4:38 PM shveta malik <shveta.malik@gmail.com>
wrote:On Fri, Jul 25, 2025 at 4:38 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:The V53-0001 also includes Shveta's comments in [1].
Thanks, I have not yet completed the review, but please find a few
comments on 001:1) IsIndexUsableForFindingDeletedTuple() We first have: + /* + * A frozen transaction ID indicates that it must fall behind the + conflict + * detection slot.xmin. + */ + if (HeapTupleHeaderXminFrozen(index_tuple->t_data)) + return true;thent his:
+ index_xmin = HeapTupleHeaderGetRawXmin(index_tuple->t_data);Shall we use HeapTupleHeaderGetXmin() instead of above 2? We can check
if xid returned by HeapTupleHeaderGetXmin() is FrozenTransactionId or
normal one and then do further processing.
I have simplified the code to avoid unnecessary checks and added generic
error handling and proper resource release, which was overlooked in the
previous version.
2) Both FindRecentlyDeletedTupleInfoByIndex and FindRecentlyDeletedTupleInfoSeq() has: + /* Exit early if the commit timestamp data is not available */ if + (!track_commit_timestamp) return false;We shall either move this check to FindDeletedTupleInLocalRel() or in
the caller of FindDeletedTupleInLocalRel() where we check
'MySubscription->retaindeadtuples'. Moving to the caller of
FindDeletedTupleInLocalRel() looks better as there is no need to call
FindDeletedTupleInLocalRel itself if pre-conditions are not met.
Moved.
3) FindRecentlyDeletedTupleInfoSeq(): + * 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. + */ Shall we extend the last line to: where the target row for an update cannot be found and index scan can not be used.
Changed.
4)
catalogs.sgml:
+ If true, he detection of <xref
+ linkend="conflict-update-deleted"/> is
he --> the
Fixed.
5)
FindRecentlyDeletedTupleInfoSeq():+ /* Get the cutoff xmin for HeapTupleSatisfiesVacuum */ oldestxmin = + GetOldestNonRemovableTransactionId(rel);Another point is which xid should be used as threshold in
HeapTupleSatisfiesVacuum() to decide if tuple is DEAD or RECENTLY-DEAD
for update_deleted case? Currently we are using
GetOldestNonRemovableTransactionId() but the xid returned here could be
older than pg_conflict_detection's xmin in presence of other logical slots which
have older effective_xmin. Shall we use pg_conflict_detection's xmin instead
as threshold or worker's oldest_nonremovable_xid?
I thought both approaches are fine in terms of correctness, because even if the
apply worker will use the outdated xmin to determine the recently dead tuples,
user can still compare the timestamp to choose the correct resolution.
OTOH, I agree that it would be better to use a stricter rule, so I changed to
use the apply worker's oldest_nonremoable_xid to determine recently dead tuples.
To be consistent, I also used this xid instead of conflict detection slot.xmin to
determine whether an index is usable or not.
This is the V54 patch set, with only patch 0001 updated to address the latest
comments.
Patch 0002 includes separate code for statistics but must be applied together
with 0001 to pass the regression tests. This is because the current
implementation assumes the number of conflict type in statistics matches the
general ConflictType enum; otherwise, it may access an invalid memory area
during stat collection.
Best Regards,
Hou zj
Attachments:
v54-0004-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v54-0004-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 0556fc3f2d940395357dd0336b86bc00afa7a53d Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:41:37 +0800
Subject: [PATCH v54 4/4] 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 | 15 ++++-
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
src/backend/utils/adt/pg_upgrade_support.c | 2 +-
src/include/replication/logicallauncher.h | 2 +-
5 files changed, 63 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0d6616857e7..e616381e44a 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5425,7 +5425,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_dead_tuples</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0e785b1a8c..50d52cbe77f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1252,7 +1252,7 @@ ApplyLauncherMain(Datum main_arg)
* applying remote changes that occurred before the
* subscription was enabled.
*/
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(!sub->enabled);
}
if (!sub->enabled)
@@ -1289,6 +1289,12 @@ ApplyLauncherMain(Datum main_arg)
{
can_advance_xmin = false;
stop_retention = false;
+
+ /*
+ * Re-create the slot if it has been invalidated, as retention
+ * has now resumed.
+ */
+ CreateConflictDetectionSlot(true);
}
/*
@@ -1515,10 +1521,15 @@ invalidate_conflict_slot(void)
* conflict detection, if not yet.
*/
void
-CreateConflictDetectionSlot(void)
+CreateConflictDetectionSlot(bool recreate_if_invalid)
{
TransactionId xmin_horizon;
+ /* Drop the invalidated slot and re-create it if requested */
+ if (recreate_if_invalid && MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d044c84ac36..c7331c4a596 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -592,6 +592,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4293,10 +4295,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4578,6 +4576,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4585,6 +4602,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4627,9 +4645,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_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_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4659,19 +4676,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index a4f8b4faa90..e20fc44adda 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -423,7 +423,7 @@ binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
{
CHECK_IS_BINARY_UPGRADE;
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(false);
ReplicationSlotRelease();
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 6e3007db5f0..5052c394c8f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -29,7 +29,7 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
-extern void CreateConflictDetectionSlot(void);
+extern void CreateConflictDetectionSlot(bool recreate_if_invalid);
extern bool IsLogicalLauncher(void);
--
2.50.1.windows.1
v54-0001-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v54-0001-Support-the-conflict-detection-for-update_delete.patchDownload
From 90fba21f78a75c91148775a55370ea678a30d24e Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 29 Jul 2025 12:18:24 +0800
Subject: [PATCH v54] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 16 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
src/backend/executor/execReplication.c | 258 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++
src/backend/replication/logical/worker.c | 160 +++++++++++--
src/include/executor/executor.h | 14 +-
src/include/replication/conflict.h | 3 +
src/include/replication/worker_internal.h | 5 +-
src/test/subscription/t/035_conflicts.pl | 65 +++++-
10 files changed, 518 insertions(+), 33 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 97f547b3cc4..da8a7882580 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretaindeadtuples</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index fcac55aefe6..b75777c6e8d 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1775,6 +1775,22 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ The tuple to be updated was recently deleted by another origin. The update
+ will simply be skipped in this scenario. Note that this conflict can only
+ be detected when
+ <link linkend="guc-track-commit-timestamp"><varname>track_commit_timestamp</varname></link>
+ and <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated, only a <literal>update_missing</literal> conflict will
+ arise. Additionally, if the tuple was deleted by the same origin, an
+ <literal>update_missing</literal> conflict will arise.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-origin-differs" xreflabel="update_origin_differs">
<term><literal>update_origin_differs</literal></term>
<listitem>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index b8cd15f3280..e847e39fa84 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index f262e7a66f7..61d51939b55 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 @@ retry:
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 @@ retry:
/*
* 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 @@ retry:
/* 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,242 @@ BuildConflictIndexInfo(ResultRelInfo *resultRelInfo, Oid conflictindex)
}
}
+/*
+ * If the tuple is identified as dead and was deleted by a transaction with a
+ * more recent commit timestamp, update the transaction ID, deletion time, and
+ * origin information associated with this tuple.
+ */
+static void
+update_recent_dead_tuple_info(TupleTableSlot *scanslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ bool recently_dead = false;
+ TransactionId xmax;
+ TimestampTz localts;
+ RepOriginId localorigin;
+
+ hslot = (BufferHeapTupleTableSlot *) scanslot;
+
+ 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 worker.c for details).
+ */
+ if (HeapTupleSatisfiesVacuum(tuple, oldestxmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ recently_dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!recently_dead)
+ return;
+
+ xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data);
+ if (!TransactionIdIsValid(xmax))
+ return;
+
+ /* 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;
+ }
+}
+
+/*
+ * 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.
+ *
+ * 'oldestxmin' serves as a cutoff transaction ID. Tuples deleted by transaction
+ * IDs greater than or equal to 'oldestxmin' are considered recently dead.
+ *
+ * We scan all matching dead tuples in the relation to find the most recently
+ * deleted one, rather than stopping at the first match. This is because only
+ * the latest deletion information is relevant for resolving conflicts.
+ *
+ * For example, consider two dead tuples on the subscriber, which can occur when
+ * a row is deleted, re-inserted, and deleted again:
+ *
+ * - (pk, 1) - deleted at 9:00,
+ * - (pk, 1) - deleted at 9:02,
+ *
+ * With a remote update (pk, 1) -> (pk, 2) timestamped at 9:01.
+ *
+ * If the first deleted tuple scanned is the older one (9:00), returning only
+ * this outdated tuple may lead users to wrongly apply the remote update using a
+ * last-update-win strategy. The appropriate action is to skip the remote
+ * update, recognizing the more recent deletion at 9:02. See comments atop
+ * worker.c for details.
+ *
+ * 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, and it
+ * should be used only when the index scans could miss deleted tuples, such as
+ * when an index has been re-indexed or re-created using CONCURRENTLY option
+ * 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 and no usable index is
+ * found.
+ */
+bool
+FindRecentlyDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ 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;
+
+ /*
+ * When the relation has an replica identity key or primary key that is
+ * not usable (see IsIndexUsableForFindingDeletedTuple), necessitating a
+ * full table scan, it is unnecessary to match the full tuple value. This
+ * is because the remote tuple may not contain all column values and using
+ * these index are sufficient to locate the target tuple (see
+ * logicalrep_rel_mark_updatable). So, we only compare indexed column
+ * values using the bitmap, which we pass to 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. Tuples from transactions
+ * not yet committed or those just committed prior to the scan are
+ * excluded in update_recent_dead_tuple_info().
+ */
+ scan = table_beginscan(rel, SnapshotAny, 0, NULL);
+ scanslot = table_slot_create(rel, NULL);
+
+ table_rescan(scan, NULL);
+
+ /* Try to find the tuple */
+ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot))
+ {
+ if (!tuples_equal(scanslot, searchslot, eq, indexbitmap))
+ continue;
+
+ update_recent_dead_tuple_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return *delete_time != 0;
+}
+
+/*
+ * Similar to FindRecentlyDeletedTupleInfoSeq() but using index scan to locate
+ * the deleted tuple.
+ */
+bool
+FindRecentlyDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ Relation idxrel;
+ ScanKeyData skey[INDEX_MAX_KEYS];
+ int skey_attoff;
+ IndexScanDesc scan;
+ TupleTableSlot *scanslot;
+ TypeCacheEntry **eq = NULL;
+ bool isIdxSafeToSkipDuplicates;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ Assert(OidIsValid(idxoid));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ isIdxSafeToSkipDuplicates = (GetRelationIdentityOrPK(rel) == idxoid);
+
+ scanslot = table_slot_create(rel, NULL);
+
+ idxrel = index_open(idxoid, RowExclusiveLock);
+
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /*
+ * Start an index scan using SnapshotAny to identify dead tuples that are
+ * not visible under a standard MVCC snapshot. Tuples from transactions
+ * not yet committed or those just committed prior to the scan are
+ * excluded in update_recent_dead_tuple_info().
+ */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, 0);
+
+ index_rescan(scan, skey, skey_attoff, NULL, 0);
+
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ /*
+ * Avoid expensive equality check if the index is primary key or
+ * replica identity index.
+ */
+ if (!isIdxSafeToSkipDuplicates)
+ {
+ if (eq == NULL)
+ eq = palloc0(sizeof(*eq) * scanslot->tts_tupleDescriptor->natts);
+
+ if (!tuples_equal(scanslot, searchslot, eq, NULL))
+ continue;
+ }
+
+ update_recent_dead_tuple_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ index_endscan(scan);
+
+ index_close(idxrel, NoLock);
+
+ 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 97c4e26b586..24d0b4ada4d 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 b59221c4d06..4cbdbe9e9fc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -138,9 +138,9 @@
* Each apply worker that enabled retain_dead_tuples option maintains a
* non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
* prevent dead rows from being removed prematurely when the apply worker still
- * needs them to detect conflicts reliably. This helps to retain the required
- * commit_ts module information, which further helps to detect
- * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * needs them to detect update_deleted conflicts. Additionally, this helps to
+ * retain the required commit_ts module information, which further helps to
+ * detect update_origin_differs and delete_origin_differs conflicts reliably, as
* otherwise, vacuum freeze could remove the required information.
*
* The logical replication launcher manages an internal replication slot named
@@ -185,10 +185,10 @@
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -576,6 +576,12 @@ static bool FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel
Oid localidxoid,
TupleTableSlot *remoteslot,
TupleTableSlot **localslot);
+static bool FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid,
+ TupleTableSlot *remoteslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
static void apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *remoteslot,
LogicalRepTupleData *newtup,
@@ -2912,17 +2918,28 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retaindeadtuples &&
+ FindDeletedTupleInLocalRel(localrel, localindexoid, 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. */
@@ -3142,6 +3159,93 @@ FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel,
return found;
}
+/*
+ * Determine whether the index can reliably locate the deleted tuple in the
+ * local relation.
+ *
+ * An index may exclude deleted tuples if it was re-indexed or re-created using
+ * the CONCURRENTLY option during change application. Therefore, an index is
+ * considered usable only if the oldest_nonremovable_xid is greater than the
+ * index tuple's xmin. This ensures that any tuples deleted prior to the index
+ * creation or re-indexing are not relevant for conflict detection in the
+ * current apply worker.
+ *
+ * Note that this might also exclude indexes that are updated due to other
+ * operations or without the CONCURRENTLY option. However, this is generally
+ * acceptable, as both the DDL commands that modify indexes and the need to scan
+ * dead tuples for the update_deleted are relatively rare occurrences.
+ */
+static bool
+IsIndexUsableForFindingDeletedTuple(Oid localindexoid)
+{
+ HeapTuple index_tuple;
+ TransactionId index_xmin;
+
+ index_tuple = SearchSysCache1(INDEXRELID, ObjectIdGetDatum(localindexoid));
+
+ if (!HeapTupleIsValid(index_tuple)) /* should not happen */
+ elog(ERROR, "cache lookup failed for index %u", localindexoid);
+
+ /*
+ * No need to check for a frozen transaction ID, as
+ * TransactionIdPrecedes() manages it internally, treating it as falling
+ * behind the oldest_nonremovable_xid.
+ */
+ index_xmin = HeapTupleHeaderGetXmin(index_tuple->t_data);
+
+ ReleaseSysCache(index_tuple);
+
+ return TransactionIdPrecedes(index_xmin,
+ MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
+/*
+ * Try to find a deleted tuple in the local relation that matching the values of
+ * the tuple received from the publication side (in 'remoteslot'). The function
+ * uses either replica identity index, primary key, index or if needed,
+ * sequential scan.
+ *
+ * Return true if found the deleted tuple. The transaction ID, commit timestamp,
+ * and origin of the transaction for the deletion, if found, are
+ * stored in '*delete_xid', '*delete_origin', and '*delete_time' respectively.
+ */
+static bool
+FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid, TupleTableSlot *remoteslot,
+ TransactionId *delete_xid, RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TransactionId oldestxmin;
+
+ /*
+ * Rather than using the conflict detection slot.xmin or invoking
+ * GetOldestNonRemovableTransactionId(), we directly use the
+ * oldest_nonremovable_xid maintained by this apply worker to identify
+ * recently deleted dead tuples for conflict detection. The
+ * oldest_nonremovable_xid is expected to be greater than or equal to
+ * other values and is more straightforward to minimize the range of dead
+ * tuples of interest for the current apply worker.
+ */
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+
+ Assert(TransactionIdIsValid(oldestxmin));
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ if (OidIsValid(localidxoid) &&
+ IsIndexUsableForFindingDeletedTuple(localidxoid))
+ return FindRecentlyDeletedTupleInfoByIndex(localrel, localidxoid,
+ remoteslot, oldestxmin,
+ delete_xid, delete_origin,
+ delete_time);
+ else
+ return FindRecentlyDeletedTupleInfoSeq(localrel, remoteslot,
+ oldestxmin, delete_xid,
+ delete_origin, delete_time);
+}
+
/*
* This handles insert, update, delete on a partitioned table.
*/
@@ -3260,18 +3364,31 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ if (MySubscription->retaindeadtuples &&
+ FindDeletedTupleInLocalRel(partrel,
+ part_entry->localindexoid,
+ 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;
}
@@ -4172,8 +4289,8 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4374,10 +4491,11 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop worker.c.
+ * required to detect update_deleted reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop
+ * worker.c.
*
* XXX Consider waiting for the publisher's clock to catch up with the
* subscriber's before proceeding to the next phase.
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 104b059544d..117c8a5527a 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"
@@ -759,7 +760,18 @@ extern bool RelationFindReplTupleByIndex(Relation rel, Oid idxoid,
TupleTableSlot *outslot);
extern bool RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode,
TupleTableSlot *searchslot, TupleTableSlot *outslot);
-
+extern bool FindRecentlyDeletedTupleInfoSeq(Relation rel,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
+extern bool FindRecentlyDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ 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 6c59125f256..cbd9656a60a 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0c7b8440a61..42f4fed38ff 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * 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
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 976d53a870e..0a4968f359d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -270,6 +273,8 @@ $node_A->psql('postgres',
###############################################################################
# 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
@@ -288,6 +293,8 @@ $node_A->poll_query_until('postgres',
"SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
);
+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;");
@@ -299,10 +306,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
@@ -324,6 +351,40 @@ ok( $stderr =~
qr/1 removed, 1 remain, 0 are dead but not yet removable/,
'the deleted column is removed');
+###############################################################################
+# Check that dead tuples can be found through a full table sequential scan
+###############################################################################
+
+# Drop the primary key from tab on node A and set REPLICA IDENTITY to FULL to
+# enforce sequential scanning of the table.
+$node_A->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_B->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_A->safe_psql('postgres', "ALTER TABLE tab DROP CONSTRAINT tab_pkey;");
+
+# Disable the logical replication from node B to node A
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE");
+
+# Wait for the apply worker to stop
+$node_A->poll_query_until('postgres',
+ "SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
+);
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 4 WHERE a = 2;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 2;");
+
+$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 \(2, 4\); replica identity full \(2, 2\)/,
+ 'update target row was deleted in tab');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.31.1
v54-0002-Collect-statistics-for-update_deleted-conflicts.patchapplication/octet-stream; name=v54-0002-Collect-statistics-for-update_deleted-conflicts.patchDownload
From 31dc2205a71c3585af9a7810f34df5e575ab6c49 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 29 Jul 2025 12:19:37 +0800
Subject: [PATCH v54 2/4] Collect statistics for update_deleted conflicts
---
doc/src/sgml/monitoring.sgml | 11 +++++++++++
src/backend/catalog/system_views.sql | 1 +
src/backend/utils/adt/pgstatfuncs.c | 18 ++++++++++--------
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 3 ++-
5 files changed, 27 insertions(+), 12 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 823afe1b30b..3d02400c870 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2199,6 +2199,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was recently deleted by another
+ source during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_origin_differs</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f6eca09ee15..e1470e48bd6 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1397,6 +1397,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/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..2a084d3f3f0 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 3ee8fed7e53..1a3b17ea112 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dce8c672b40..56704426431 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2177,6 +2177,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,
@@ -2185,7 +2186,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,
--
2.50.1.windows.1
v54-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v54-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From be1bbda2c72ba4032419e836f57e9ee62d471914 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v54 3/4] 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_dead_tuples 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_dead_tuples 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_dead_tuples 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_dead_tuples' 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_dead_tuples 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 | 43 ++++++
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 | 120 +++++++++++++++--
src/backend/replication/logical/worker.c | 126 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 348 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..0d6616857e7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_dead_tuples</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3d02400c870..3d56a07ad96 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 4187191ea74..27e55bb0f35 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -3007,6 +3007,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index e1470e48bd6..d4eabeffaa7 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 742d9ba68e9..c0e785b1a8c 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;
@@ -100,9 +103,11 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -465,6 +470,8 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1181,6 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1223,6 +1231,13 @@ ApplyLauncherMain(Datum main_arg)
*/
can_advance_xmin &= sub->enabled;
+ /*
+ * Stop the conflict information retention only if all workers
+ * for subscriptions with retain_dead_tuples enabled have
+ * requested it.
+ */
+ stop_retention &= sub->enabled;
+
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1254,8 +1269,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin,
+ &stop_retention);
/* worker is running already */
continue;
@@ -1265,10 +1281,15 @@ ApplyLauncherMain(Datum main_arg)
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_dead_tuples are
* running, disabling the further computation of the minimum
- * nonremovable xid.
+ * nonremovable xid. Similarly, stop the conflict information
+ * retention only if all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
*/
if (sub->retaindeadtuples)
+ {
can_advance_xmin = false;
+ stop_retention = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1314,13 +1335,16 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain dead tuples. Otherwise, if required,
- * advance the slot's xmin to protect dead tuples required for the
- * conflict detection.
+ * Invalidate the slot if requested by all apply workers. Or, drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain dead tuples. Otherwise, if required, advance
+ * the slot's xmin to protect dead tuples required for the conflict
+ * detection.
*/
if (MyReplicationSlot)
{
+ if (retain_dead_tuples && stop_retention)
+ invalidate_conflict_slot();
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
@@ -1358,11 +1382,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention)
{
TransactionId nonremovable_xid;
+ bool stop_conflict_info_retention;
Assert(worker != NULL);
@@ -1374,8 +1403,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_dead_tuples enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1436,6 +1479,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_dead_tuples enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1516,7 +1590,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;
@@ -1593,6 +1667,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->subretaindeadtuples &&
+ !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 cb6de3d9979..d044c84ac36 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4072,7 +4077,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4287,6 +4293,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4430,6 +4440,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4511,6 +4528,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4556,12 +4594,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4569,9 +4616,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4604,6 +4708,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8605776ad86..6a7d2076385 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"},
};
/*
@@ -1665,6 +1666,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();
}
@@ -1783,6 +1789,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s,
}
}
+ if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION)
+ {
+ if (IsSlotForConflictCheck(NameStr(s->data.name)))
+ return RS_INVAL_CONFLICT_RETENTION_DURATION;
+ }
+
return RS_INVAL_NONE;
}
@@ -2044,6 +2056,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 d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 1a3b17ea112..b084ba5a669 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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 e8fc342d1a9..2f459404c9e 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 42f4fed38ff..d1e755b37cf 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining conflict
+ * information. This is used only when retain_dead_tuples 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 56704426431..33253fb2cc6 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2169,9 +2169,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 0a4968f359d..ba323b5109e 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Monday, July 28, 2025 5:54 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Jul 25, 2025 at 4:38 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Right, I think it makes sense to do with the index scan when the
index's xmin is less than the conflict detection xmin, as that can
ensure that all the tuples deleted before the index creation or
re-indexing are irrelevant for conflict detection.I have implemented in the V53 patch set and improved the test to
verify both index and seq scan for dead tuples.Thanks. Following are a few comments on 0001 patch:
1. --- a/src/backend/catalog/system_views.sql +++ b/src/backend/catalog/system_views.sql @@ -1397,6 +1397,7 @@ CREATE VIEW pg_stat_subscription_stats AS ss.apply_error_count, ss.sync_error_count, ss.confl_insert_exists, + ss.confl_update_deleted, … Datum pg_stat_get_subscription_stats(PG_FUNCTION_ARGS) { -#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 11 +#define PG_STAT_GET_SUBSCRIPTION_STATS_COLS 12Can we consider splitting stats into a separate patch? It will help us to first
focus on core functionality of detecting update_delete conflict.
OK, splitted.
2. 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.Here we should add at end "and no usable index is found"
Added.
3. + * We scan all matching dead tuples in the relation to find the most + recently + * deleted one, rather than stopping at the first match. This is + because only + * the latest deletion information is relevant for resolving conflicts. + * Returning solely the first, potentially outdated tuple can lead + users to + * mistakenly apply remote changes using a last-update-win strategy, even when a + * more recent deleted tuple is available. See comments atop worker.c + for + * details.I think we can share a short example of cases when this can happen.
Added the comments.
And probably a test which will fail if the user only fetches the first dead tuple?
I am still thinking how to write a test that can distinguish two different
dead tuples and will add in next version if possible.
4.
executor\execReplication.c(671) : warning C4700: uninitialized local variable
'eq' usedPlease fix this warning.
Fixed. Sorry for the miss.
5. + /* Build scan key. */ + skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot); + + /* Start an index scan. */ + scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, + 0);While scanning with SnapshotAny, isn't it possible that we find some tuple for
which the xact is still not committed or are inserted successfully just before the
scan is started?I think such tuples shouldn't be considered for giving update_deleted.
It seems the patch will handle it later during
update_recent_dead_tuple_info() where it uses following check: "if
(HeapTupleSatisfiesVacuum(tuple, oldestxmin, buf) ==
HEAPTUPLE_RECENTLY_DEAD)", is my understanding correct? If so, we
should add some comments for it.
Added.
6. FindRecentlyDeletedTupleInfoSeq() { … + /* 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); … ... + if (!tuples_equal(scanslot, searchslot, eq, indexbitmap)) continue;We don't do any such thing in RelationFindReplTupleSeq(), so, if we do
something differently here, it should be explained in the comments.
Added.
Best Regards,
Hou zj
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
This is the V54 patch set, with only patch 0001 updated to address the latest
comments.
Thanks for the patch.
While performing tests on the latest patch, I found an assert in
tablesync worker in FindDeletedTupleInLocalRel (see
Assert(TransactionIdIsValid(oldestxmin))). Logs at [1]TRAP: failed Assert("TransactionIdIsValid(oldestxmin)"), File: "worker.c", Line: 3237, PID: 96003 postgres: logical replication tablesync worker for subscription 16394 sync 16469 (ExceptionalCondition+0xbb)[0x619c4117031b] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x65f4ca)[0x619c40ea94ca] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x65fa23)[0x619c40ea9a23] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x65e860)[0x619c40ea8860] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (apply_dispatch+0xa2)[0x619c40eaa725] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x660db9)[0x619c40eaadb9] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (start_apply+0x81)[0x619c40eace64] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x659f1f)[0x619c40ea3f1f] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (TablesyncWorkerMain+0x33)[0x619c40ea3f69].
It seems table sync worker is trying to apply changes and going to
update-deleted conflict detection patch but is not able to find
MyLogicalRepWorker->oldest_nonremovable_xid as this xid is set only
for apply-worker.
[1]: TRAP: failed Assert("TransactionIdIsValid(oldestxmin)"), File: "worker.c", Line: 3237, PID: 96003 postgres: logical replication tablesync worker for subscription 16394 sync 16469 (ExceptionalCondition+0xbb)[0x619c4117031b] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x65f4ca)[0x619c40ea94ca] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x65fa23)[0x619c40ea9a23] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x65e860)[0x619c40ea8860] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (apply_dispatch+0xa2)[0x619c40eaa725] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x660db9)[0x619c40eaadb9] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (start_apply+0x81)[0x619c40eace64] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (+0x659f1f)[0x619c40ea3f1f] postgres: logical replication tablesync worker for subscription 16394 sync 16469 (TablesyncWorkerMain+0x33)[0x619c40ea3f69]
TRAP: failed Assert("TransactionIdIsValid(oldestxmin)"), File:
"worker.c", Line: 3237, PID: 96003
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (ExceptionalCondition+0xbb)[0x619c4117031b]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (+0x65f4ca)[0x619c40ea94ca]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (+0x65fa23)[0x619c40ea9a23]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (+0x65e860)[0x619c40ea8860]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (apply_dispatch+0xa2)[0x619c40eaa725]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (+0x660db9)[0x619c40eaadb9]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (start_apply+0x81)[0x619c40eace64]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (+0x659f1f)[0x619c40ea3f1f]
postgres: logical replication tablesync worker for subscription 16394
sync 16469 (TablesyncWorkerMain+0x33)[0x619c40ea3f69]
thanks
Shveta
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
This is the V54 patch set, with only patch 0001 updated to address the latest
comments.
Few minor comments:
1.
/* 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,
/* The updated row value violates unique constraint */
CT_UPDATE_EXISTS,
/* The row to be updated is missing */
CT_UPDATE_MISSING,
Is there a reason to keep CT_UPDATE_DELETED before
CT_UPDATE_ORIGIN_DIFFERS? I mean why not keep it just before
CT_UPDATE_MISSING on the grounds that they are always handled
together?
2. Will it be better to name FindRecentlyDeletedTupleInfoByIndex as
RelationFindDeletedTupleInfoByIndex to make it similar to existing
function RelationFindReplTupleByIndex? If you agree then make a
similar change for FindRecentlyDeletedTupleInfoSeq as well.
Apart from above, please find a number of comment edits and other
cosmetic changes in the attached.
--
With Regards,
Amit Kapila.
Attachments:
v54-amit-1.patch.txttext/plain; charset=US-ASCII; name=v54-amit-1.patch.txtDownload
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index 61d51939b55..1bf8fac7bea 100644
--- a/src/backend/executor/execReplication.c
+++ b/src/backend/executor/execReplication.c
@@ -470,16 +470,17 @@ BuildConflictIndexInfo(ResultRelInfo *resultRelInfo, Oid conflictindex)
}
/*
- * If the tuple is identified as dead and was deleted by a transaction with a
- * more recent commit timestamp, update the transaction ID, deletion time, and
- * origin information associated with this tuple.
+ * If the tuple is recently dead and was deleted by a transaction with a newer
+ * commit timestamp than previously recorded, update the associated transaction
+ * ID, commit time, and origin. This helps ensure that conflict detection uses
+ * the most recent and relevant deletion metadata.
*/
static void
-update_recent_dead_tuple_info(TupleTableSlot *scanslot,
- TransactionId oldestxmin,
- TransactionId *delete_xid,
- TimestampTz *delete_time,
- RepOriginId *delete_origin)
+update_most_recent_deletion_info(TupleTableSlot *scanslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
{
BufferHeapTupleTableSlot *hslot;
HeapTuple tuple;
@@ -532,37 +533,31 @@ update_recent_dead_tuple_info(TupleTableSlot *scanslot,
* returns the transaction ID, origin, and commit timestamp of the transaction
* that deleted this tuple.
*
- * 'oldestxmin' serves as a cutoff transaction ID. Tuples deleted by transaction
- * IDs greater than or equal to 'oldestxmin' are considered recently dead.
+ * 'oldestxmin' acts as a cutoff transaction ID. Tuples deleted by transactions
+ * with IDs >= 'oldestxmin' are considered recently dead and are eligible for
+ * conflict detection.
*
- * We scan all matching dead tuples in the relation to find the most recently
- * deleted one, rather than stopping at the first match. This is because only
- * the latest deletion information is relevant for resolving conflicts.
+ * Instead of stopping at the first match, we scan all matching dead tuples to
+ * identify most recent deletion. This is crucial because only the latest
+ * deletion is relevant for resolving conflicts.
*
- * For example, consider two dead tuples on the subscriber, which can occur when
- * a row is deleted, re-inserted, and deleted again:
+ * For example, consider a scenario on the subscriber where a row is deleted,
+ * re-inserted, and then deleted again only on the subscriber:
*
* - (pk, 1) - deleted at 9:00,
* - (pk, 1) - deleted at 9:02,
*
- * With a remote update (pk, 1) -> (pk, 2) timestamped at 9:01.
+ * Now, a remote update arrives: (pk, 1) -> (pk, 2), timestamped at 9:01.
*
- * If the first deleted tuple scanned is the older one (9:00), returning only
- * this outdated tuple may lead users to wrongly apply the remote update using a
- * last-update-win strategy. The appropriate action is to skip the remote
- * update, recognizing the more recent deletion at 9:02. See comments atop
- * worker.c for details.
+ * If we mistakenly return the older deletion (9:00), the system may wrongly
+ * apply the remote update using a last-update-wins strategy. Instead, we must
+ * recognize the more recent deletion at 9:02 and skip the update. See
+ * comments atop worker.c for details. Note, as of now, conflict resolution
+ * is not implemented. Consequently, the system may incorrectly report the
+ * older tuple as the conflicted one, leading to misleading results.
*
- * 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, and it
- * should be used only when the index scans could miss deleted tuples, such as
- * when an index has been re-indexed or re-created using CONCURRENTLY option
- * 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 and no usable index is
- * found.
+ * The commit timestamp of the deleting transaction is used to determine which
+ * tuple was deleted most recently.
*/
bool
FindRecentlyDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
@@ -584,13 +579,13 @@ FindRecentlyDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
*delete_time = 0;
/*
- * When the relation has an replica identity key or primary key that is
- * not usable (see IsIndexUsableForFindingDeletedTuple), necessitating a
- * full table scan, it is unnecessary to match the full tuple value. This
- * is because the remote tuple may not contain all column values and using
- * these index are sufficient to locate the target tuple (see
- * logicalrep_rel_mark_updatable). So, we only compare indexed column
- * values using the bitmap, which we pass to tuples_equal().
+ * If the relation has a replica identity key or a primary key that is
+ * unusable for locating deleted tuples (see
+ * IsIndexUsableForFindingDeletedTuple), a full table scan becomes
+ * necessary. In such cases, comparing the entire tuple is not required,
+ * since the remote tuple might not include all column values. Instead, the
+ * indexed columns alone are suffcient to identify the target tuple (see
+ * logicalrep_rel_mark_updatable).
*/
indexbitmap = RelationGetIndexAttrBitmap(rel,
INDEX_ATTR_BITMAP_IDENTITY_KEY);
@@ -606,7 +601,7 @@ FindRecentlyDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
* Start a heap scan using SnapshotAny to identify dead tuples that are
* not visible under a standard MVCC snapshot. Tuples from transactions
* not yet committed or those just committed prior to the scan are
- * excluded in update_recent_dead_tuple_info().
+ * excluded in update_most_recent_deletion_info().
*/
scan = table_beginscan(rel, SnapshotAny, 0, NULL);
scanslot = table_slot_create(rel, NULL);
@@ -619,8 +614,8 @@ FindRecentlyDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
if (!tuples_equal(scanslot, searchslot, eq, indexbitmap))
continue;
- update_recent_dead_tuple_info(scanslot, oldestxmin, delete_xid,
- delete_time, delete_origin);
+ update_most_recent_deletion_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
}
table_endscan(scan);
@@ -670,7 +665,7 @@ FindRecentlyDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
* Start an index scan using SnapshotAny to identify dead tuples that are
* not visible under a standard MVCC snapshot. Tuples from transactions
* not yet committed or those just committed prior to the scan are
- * excluded in update_recent_dead_tuple_info().
+ * excluded in update_most_recent_deletion_info().
*/
scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, 0);
@@ -692,8 +687,8 @@ FindRecentlyDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
continue;
}
- update_recent_dead_tuple_info(scanslot, oldestxmin, delete_xid,
- delete_time, delete_origin);
+ update_most_recent_deletion_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
}
index_endscan(scan);
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f1e56d84bb3..45ec3ad593d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -2921,6 +2921,10 @@ apply_handle_update_internal(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
+ /*
+ * Detecting whether the tuple was recently deleted or never existed is
+ * crucial to avoid misleading the user during confict handling.
+ */
if (MySubscription->retaindeadtuples &&
FindDeletedTupleInLocalRel(localrel, localindexoid, remoteslot,
&conflicttuple.xmin,
@@ -3163,17 +3167,17 @@ FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel,
* Determine whether the index can reliably locate the deleted tuple in the
* local relation.
*
- * An index may exclude deleted tuples if it was re-indexed or re-created using
- * the CONCURRENTLY option during change application. Therefore, an index is
- * considered usable only if the oldest_nonremovable_xid is greater than the
- * index tuple's xmin. This ensures that any tuples deleted prior to the index
- * creation or re-indexing are not relevant for conflict detection in the
- * current apply worker.
+ * An index may exclude deleted tuples if it was re-indexed or re-created
+ * during change application. Therefore, an index is considered usable only
+ * if the oldest_nonremovable_xid is greater than the index tuple's xmin.
+ * This ensures that any tuples deleted prior to the index creation or
+ * re-indexing are not relevant for conflict detection in the current apply
+ * worker.
*
- * Note that this might also exclude indexes that are updated due to other
- * operations or without the CONCURRENTLY option. However, this is generally
- * acceptable, as both the DDL commands that modify indexes and the need to scan
- * dead tuples for the update_deleted are relatively rare occurrences.
+ * Note that indexes may also be excluded if they were modified by other DDL
+ * operations, such as ALTER INDEX. However, this is acceptable, as the
+ * likelihood of such DDL changes coinciding with the need to scan dead
+ * tuples for the update_deleted is low.
*/
static bool
IsIndexUsableForFindingDeletedTuple(Oid localindexoid)
@@ -3200,14 +3204,14 @@ IsIndexUsableForFindingDeletedTuple(Oid localindexoid)
}
/*
- * Try to find a deleted tuple in the local relation that matching the values of
- * the tuple received from the publication side (in 'remoteslot'). The function
- * uses either replica identity index, primary key, index or if needed,
- * sequential scan.
+ * Attempts to locate a deleted tuple in the local relation that matches the
+ * values of the tuple received from the publication side (in 'remoteslot').
+ * The search is performed using either the replica identity index, primary
+ * key, other available index, or a sequential scan if necessary.
*
- * Return true if found the deleted tuple. The transaction ID, commit timestamp,
- * and origin of the transaction for the deletion, if found, are
- * stored in '*delete_xid', '*delete_origin', and '*delete_time' respectively.
+ * Returns true if the deleted tuple is found. If found, the transaction ID,
+ * origin, and commit timestamp of the deletion are stored in '*delete_xid',
+ * '*delete_origin', and '*delete_time' respectively.
*/
static bool
FindDeletedTupleInLocalRel(Relation localrel,
@@ -3218,13 +3222,12 @@ FindDeletedTupleInLocalRel(Relation localrel,
TransactionId oldestxmin;
/*
- * Rather than using the conflict detection slot.xmin or invoking
- * GetOldestNonRemovableTransactionId(), we directly use the
- * oldest_nonremovable_xid maintained by this apply worker to identify
- * recently deleted dead tuples for conflict detection. The
- * oldest_nonremovable_xid is expected to be greater than or equal to
- * other values and is more straightforward to minimize the range of dead
- * tuples of interest for the current apply worker.
+ * Instead of relying on slot.xmin or invoking
+ * GetOldestNonRemovableTransactionId() for conflict detection, we use the
+ * oldest_nonremovable_xid maintained by this apply worker. This value will
+ * be greater than or equal to other thresholds and provides a more direct
+ * and efficient way to identify recently deleted dead tuples relevant to
+ * the current apply worker.
*/
oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
@@ -3367,6 +3370,11 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
ConflictType type;
TupleTableSlot *newslot = localslot;
+ /*
+ * Detecting whether the tuple was recently deleted or
+ * never existed is crucial to avoid misleading the user
+ * during confict handling.
+ */
if (MySubscription->retaindeadtuples &&
FindDeletedTupleInLocalRel(partrel,
part_entry->localindexoid,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 42f4fed38ff..7c0204dd6f4 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,9 +87,9 @@ typedef struct LogicalRepWorker
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.
+ * Changes made by this transaction and subsequent ones must be preserved.
+ * This ensures that update_deleted conflicts can be accurately detected
+ * during the apply phase of logical replication by this worker.
*
* The logical replication launcher manages an internal replication slot
* named "pg_conflict_detection". It asynchronously collects this ID to
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 0a4968f359d..36aeb14c563 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -273,7 +273,7 @@ $node_A->psql('postgres',
###############################################################################
# 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
+# Also, check that an update_deleted conflict is detected when updating a row
# that was deleted by a different origin.
###############################################################################
@@ -352,7 +352,8 @@ ok( $stderr =~
'the deleted column is removed');
###############################################################################
-# Check that dead tuples can be found through a full table sequential scan
+# Ensure that the deleted tuple needed to detect an update_deleted conflict is
+# accessible via a sequential table scan.
###############################################################################
# Drop the primary key from tab on node A and set REPLICA IDENTITY to FULL to
On Thursday, July 31, 2025 5:29 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:This is the V54 patch set, with only patch 0001 updated to address the
latest comments.Few minor comments:
Thanks for the comments.
1.
/* 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,
/* The updated row value violates unique constraint */ CT_UPDATE_EXISTS,
/* The row to be updated is missing */
CT_UPDATE_MISSING,Is there a reason to keep CT_UPDATE_DELETED before
CT_UPDATE_ORIGIN_DIFFERS? I mean why not keep it just before
CT_UPDATE_MISSING on the grounds that they are always handled together?
I agree that it makes more sense to put it before update_missing, and changed it.
2. Will it be better to name FindRecentlyDeletedTupleInfoByIndex as
RelationFindDeletedTupleInfoByIndex to make it similar to existing function
RelationFindReplTupleByIndex? If you agree then make a similar change for
FindRecentlyDeletedTupleInfoSeq as well.
Yes, the suggested name looks better.
Apart from above, please find a number of comment edits and other cosmetic
changes in the attached.
Thanks, I have addressed above comments and merge the patch into 0001.
Here is V55 patch set.
Best Regards,
Hou zj
Attachments:
v55-0004-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v55-0004-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From e97aa070e763546be7e3de8388a40717bf5a3915 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:41:37 +0800
Subject: [PATCH v55 4/4] 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 | 15 ++++-
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
src/backend/utils/adt/pg_upgrade_support.c | 2 +-
src/include/replication/logicallauncher.h | 2 +-
5 files changed, 63 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0d6616857e7..e616381e44a 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5425,7 +5425,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_dead_tuples</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0e785b1a8c..50d52cbe77f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1252,7 +1252,7 @@ ApplyLauncherMain(Datum main_arg)
* applying remote changes that occurred before the
* subscription was enabled.
*/
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(!sub->enabled);
}
if (!sub->enabled)
@@ -1289,6 +1289,12 @@ ApplyLauncherMain(Datum main_arg)
{
can_advance_xmin = false;
stop_retention = false;
+
+ /*
+ * Re-create the slot if it has been invalidated, as retention
+ * has now resumed.
+ */
+ CreateConflictDetectionSlot(true);
}
/*
@@ -1515,10 +1521,15 @@ invalidate_conflict_slot(void)
* conflict detection, if not yet.
*/
void
-CreateConflictDetectionSlot(void)
+CreateConflictDetectionSlot(bool recreate_if_invalid)
{
TransactionId xmin_horizon;
+ /* Drop the invalidated slot and re-create it if requested */
+ if (recreate_if_invalid && MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 36b3446c2f9..cdb7128dda7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -592,6 +592,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4321,10 +4323,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4606,6 +4604,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4613,6 +4630,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4655,9 +4673,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_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_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4687,19 +4704,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index a4f8b4faa90..e20fc44adda 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -423,7 +423,7 @@ binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
{
CHECK_IS_BINARY_UPGRADE;
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(false);
ReplicationSlotRelease();
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 6e3007db5f0..5052c394c8f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -29,7 +29,7 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
-extern void CreateConflictDetectionSlot(void);
+extern void CreateConflictDetectionSlot(bool recreate_if_invalid);
extern bool IsLogicalLauncher(void);
--
2.50.1.windows.1
v55-0001-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v55-0001-Support-the-conflict-detection-for-update_delete.patchDownload
From 9d68ae6aea5dbca2fbfd961ef03a9f53b719c08f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 29 Jul 2025 12:18:24 +0800
Subject: [PATCH v55] 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/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 16 ++
doc/src/sgml/ref/create_subscription.sgml | 5 +-
src/backend/executor/execReplication.c | 252 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++
src/backend/replication/logical/worker.c | 176 ++++++++++++--
src/include/executor/executor.h | 14 +-
src/include/replication/conflict.h | 3 +
src/include/replication/worker_internal.h | 5 +-
src/test/subscription/t/035_conflicts.pl | 66 +++++-
10 files changed, 529 insertions(+), 33 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 97f547b3cc4..da8a7882580 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretaindeadtuples</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index fcac55aefe6..ab219e87f71 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1804,6 +1804,22 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ The tuple to be updated was recently deleted by another origin. The update
+ will simply be skipped in this scenario. Note that this conflict can only
+ be detected when
+ <link linkend="guc-track-commit-timestamp"><varname>track_commit_timestamp</varname></link>
+ and <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated, only a <literal>update_missing</literal> conflict will
+ arise. Additionally, if the tuple was deleted by the same origin, an
+ <literal>update_missing</literal> conflict will arise.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-missing" xreflabel="update_missing">
<term><literal>update_missing</literal></term>
<listitem>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index b8cd15f3280..e847e39fa84 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,8 +445,9 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
</para>
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index f262e7a66f7..9a0f7c31d4b 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"
@@ -36,7 +38,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 +223,7 @@ retry:
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 +279,13 @@ retry:
/*
* 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 +310,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 +393,7 @@ retry:
/* 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 +468,237 @@ BuildConflictIndexInfo(ResultRelInfo *resultRelInfo, Oid conflictindex)
}
}
+/*
+ * If the tuple is recently dead and was deleted by a transaction with a newer
+ * commit timestamp than previously recorded, update the associated transaction
+ * ID, commit time, and origin. This helps ensure that conflict detection uses
+ * the most recent and relevant deletion metadata.
+ */
+static void
+update_most_recent_deletion_info(TupleTableSlot *scanslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ bool recently_dead = false;
+ TransactionId xmax;
+ TimestampTz localts;
+ RepOriginId localorigin;
+
+ hslot = (BufferHeapTupleTableSlot *) scanslot;
+
+ 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 worker.c for details).
+ */
+ if (HeapTupleSatisfiesVacuum(tuple, oldestxmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ recently_dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!recently_dead)
+ return;
+
+ xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data);
+ if (!TransactionIdIsValid(xmax))
+ return;
+
+ /* 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;
+ }
+}
+
+/*
+ * 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.
+ *
+ * 'oldestxmin' acts as a cutoff transaction ID. Tuples deleted by transactions
+ * with IDs >= 'oldestxmin' are considered recently dead and are eligible for
+ * conflict detection.
+ *
+ * Instead of stopping at the first match, we scan all matching dead tuples to
+ * identify most recent deletion. This is crucial because only the latest
+ * deletion is relevant for resolving conflicts.
+ *
+ * For example, consider a scenario on the subscriber where a row is deleted,
+ * re-inserted, and then deleted again only on the subscriber:
+ *
+ * - (pk, 1) - deleted at 9:00,
+ * - (pk, 1) - deleted at 9:02,
+ *
+ * Now, a remote update arrives: (pk, 1) -> (pk, 2), timestamped at 9:01.
+ *
+ * If we mistakenly return the older deletion (9:00), the system may wrongly
+ * apply the remote update using a last-update-wins strategy. Instead, we must
+ * recognize the more recent deletion at 9:02 and skip the update. See
+ * comments atop worker.c for details. Note, as of now, conflict resolution
+ * is not implemented. Consequently, the system may incorrectly report the
+ * older tuple as the conflicted one, leading to misleading results.
+ *
+ * The commit timestamp of the deleting transaction is used to determine which
+ * tuple was deleted most recently.
+ */
+bool
+RelationFindDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ 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;
+
+ /*
+ * If the relation has a replica identity key or a primary key that is
+ * unusable for locating deleted tuples (see
+ * IsIndexUsableForFindingDeletedTuple), a full table scan becomes
+ * necessary. In such cases, comparing the entire tuple is not required,
+ * since the remote tuple might not include all column values. Instead,
+ * the indexed columns alone are suffcient to identify the target tuple
+ * (see logicalrep_rel_mark_updatable).
+ */
+ 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. Tuples from transactions
+ * not yet committed or those just committed prior to the scan are
+ * excluded in update_most_recent_deletion_info().
+ */
+ scan = table_beginscan(rel, SnapshotAny, 0, NULL);
+ scanslot = table_slot_create(rel, NULL);
+
+ table_rescan(scan, NULL);
+
+ /* Try to find the tuple */
+ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot))
+ {
+ if (!tuples_equal(scanslot, searchslot, eq, indexbitmap))
+ continue;
+
+ update_most_recent_deletion_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return *delete_time != 0;
+}
+
+/*
+ * Similar to RelationFindDeletedTupleInfoSeq() but using index scan to locate
+ * the deleted tuple.
+ */
+bool
+RelationFindDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ Relation idxrel;
+ ScanKeyData skey[INDEX_MAX_KEYS];
+ int skey_attoff;
+ IndexScanDesc scan;
+ TupleTableSlot *scanslot;
+ TypeCacheEntry **eq = NULL;
+ bool isIdxSafeToSkipDuplicates;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ Assert(OidIsValid(idxoid));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ isIdxSafeToSkipDuplicates = (GetRelationIdentityOrPK(rel) == idxoid);
+
+ scanslot = table_slot_create(rel, NULL);
+
+ idxrel = index_open(idxoid, RowExclusiveLock);
+
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /*
+ * Start an index scan using SnapshotAny to identify dead tuples that are
+ * not visible under a standard MVCC snapshot. Tuples from transactions
+ * not yet committed or those just committed prior to the scan are
+ * excluded in update_most_recent_deletion_info().
+ */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, 0);
+
+ index_rescan(scan, skey, skey_attoff, NULL, 0);
+
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ /*
+ * Avoid expensive equality check if the index is primary key or
+ * replica identity index.
+ */
+ if (!isIdxSafeToSkipDuplicates)
+ {
+ if (eq == NULL)
+ eq = palloc0(sizeof(*eq) * scanslot->tts_tupleDescriptor->natts);
+
+ if (!tuples_equal(scanslot, searchslot, eq, NULL))
+ continue;
+ }
+
+ update_most_recent_deletion_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ index_endscan(scan);
+
+ index_close(idxrel, NoLock);
+
+ 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 97c4e26b586..2fd3e8bbda5 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -29,6 +29,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_MISSING] = "delete_missing",
[CT_MULTIPLE_UNIQUE_CONFLICTS] = "multiple_unique_conflicts"
};
@@ -176,6 +177,7 @@ errcode_apply_conflict(ConflictType type)
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
+ case CT_UPDATE_DELETED:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
}
@@ -261,6 +263,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_MISSING:
appendStringInfoString(&err_detail, _("Could not find the row to be updated."));
break;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b59221c4d06..2071c1b28af 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -138,9 +138,9 @@
* Each apply worker that enabled retain_dead_tuples option maintains a
* non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
* prevent dead rows from being removed prematurely when the apply worker still
- * needs them to detect conflicts reliably. This helps to retain the required
- * commit_ts module information, which further helps to detect
- * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * needs them to detect update_deleted conflicts. Additionally, this helps to
+ * retain the required commit_ts module information, which further helps to
+ * detect update_origin_differs and delete_origin_differs conflicts reliably, as
* otherwise, vacuum freeze could remove the required information.
*
* The logical replication launcher manages an internal replication slot named
@@ -185,10 +185,10 @@
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -576,6 +576,12 @@ static bool FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel
Oid localidxoid,
TupleTableSlot *remoteslot,
TupleTableSlot **localslot);
+static bool FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid,
+ TupleTableSlot *remoteslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
static void apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *remoteslot,
LogicalRepTupleData *newtup,
@@ -2912,17 +2918,32 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ /*
+ * Detecting whether the tuple was recently deleted or never existed
+ * is crucial to avoid misleading the user during confict handling.
+ */
+ if (MySubscription->retaindeadtuples &&
+ FindDeletedTupleInLocalRel(localrel, localindexoid, 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. */
@@ -3142,6 +3163,100 @@ FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel,
return found;
}
+/*
+ * Determine whether the index can reliably locate the deleted tuple in the
+ * local relation.
+ *
+ * An index may exclude deleted tuples if it was re-indexed or re-created during
+ * change application. Therefore, an index is considered usable only if the
+ * conflict detection slot.xmin (conflict_detection_xmin) is greater than the
+ * index tuple's xmin. This ensures that any tuples deleted prior to the index
+ * creation or re-indexing are not relevant for conflict detection in the
+ * current apply worker.
+ *
+ * Note that indexes may also be excluded if they were modified by other DDL
+ * operations, such as ALTER INDEX. However, this is acceptable, as the
+ * likelihood of such DDL changes coinciding with the need to scan dead
+ * tuples for the update_deleted is low.
+ */
+static bool
+IsIndexUsableForFindingDeletedTuple(Oid localindexoid,
+ TransactionId conflict_detection_xmin)
+{
+ HeapTuple index_tuple;
+ TransactionId index_xmin;
+
+ index_tuple = SearchSysCache1(INDEXRELID, ObjectIdGetDatum(localindexoid));
+
+ if (!HeapTupleIsValid(index_tuple)) /* should not happen */
+ elog(ERROR, "cache lookup failed for index %u", localindexoid);
+
+ /*
+ * No need to check for a frozen transaction ID, as
+ * TransactionIdPrecedes() manages it internally, treating it as falling
+ * behind the oldest_nonremovable_xid.
+ */
+ index_xmin = HeapTupleHeaderGetXmin(index_tuple->t_data);
+
+ ReleaseSysCache(index_tuple);
+
+ return TransactionIdPrecedes(index_xmin, conflict_detection_xmin);
+}
+
+/*
+ * Attempts to locate a deleted tuple in the local relation that matches the
+ * values of the tuple received from the publication side (in 'remoteslot').
+ * The search is performed using either the replica identity index, primary
+ * key, other available index, or a sequential scan if necessary.
+ *
+ * Returns true if the deleted tuple is found. If found, the transaction ID,
+ * origin, and commit timestamp of the deletion are stored in '*delete_xid',
+ * '*delete_origin', and '*delete_time' respectively.
+ */
+static bool
+FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid, TupleTableSlot *remoteslot,
+ TransactionId *delete_xid, RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TransactionId oldestxmin;
+ ReplicationSlot *slot;
+
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
+
+ /*
+ * Instead of invoking GetOldestNonRemovableTransactionId() for conflict
+ * detection, we use the conflict detection slot.xmin. This value will be
+ * greater than or equal to the other threshold and provides a more direct
+ * and efficient way to identify recently deleted dead tuples relevant to
+ * the conflict detection. The oldest_nonremovable_xid is not used here,
+ * as it is maintained only by the leader apply worker and unavailable to
+ * table sync and parallel apply workers.
+ */
+ slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+
+ Assert(slot);
+
+ SpinLockAcquire(&slot->mutex);
+ oldestxmin = slot->data.xmin;
+ SpinLockRelease(&slot->mutex);
+
+ Assert(TransactionIdIsValid(oldestxmin));
+
+ if (OidIsValid(localidxoid) &&
+ IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
+ return RelationFindDeletedTupleInfoByIndex(localrel, localidxoid,
+ remoteslot, oldestxmin,
+ delete_xid, delete_origin,
+ delete_time);
+ else
+ return RelationFindDeletedTupleInfoSeq(localrel, remoteslot,
+ oldestxmin, delete_xid,
+ delete_origin, delete_time);
+}
+
/*
* This handles insert, update, delete on a partitioned table.
*/
@@ -3260,18 +3375,36 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ /*
+ * Detecting whether the tuple was recently deleted or
+ * never existed is crucial to avoid misleading the user
+ * during confict handling.
+ */
+ if (MySubscription->retaindeadtuples &&
+ FindDeletedTupleInLocalRel(partrel,
+ part_entry->localindexoid,
+ 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;
}
@@ -4172,8 +4305,8 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4374,10 +4507,11 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop worker.c.
+ * required to detect update_deleted reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop
+ * worker.c.
*
* XXX Consider waiting for the publisher's clock to catch up with the
* subscriber's before proceeding to the next phase.
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 104b059544d..a71502efeed 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"
@@ -759,7 +760,18 @@ extern bool RelationFindReplTupleByIndex(Relation rel, Oid idxoid,
TupleTableSlot *outslot);
extern bool RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode,
TupleTableSlot *searchslot, TupleTableSlot *outslot);
-
+extern bool RelationFindDeletedTupleInfoSeq(Relation rel,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
+extern bool RelationFindDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ 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 6c59125f256..21340641224 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -32,6 +32,9 @@ typedef enum
/* The updated row value violates unique constraint */
CT_UPDATE_EXISTS,
+ /* The row to be updated was deleted by a different origin */
+ CT_UPDATE_DELETED,
+
/* The row to be updated is missing */
CT_UPDATE_MISSING,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0c7b8440a61..7c0204dd6f4 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * Changes made by this transaction and subsequent ones must be preserved.
+ * This ensures that update_deleted conflicts can be accurately detected
+ * during the apply phase of logical replication by this worker.
*
* The logical replication launcher manages an internal replication slot
* named "pg_conflict_detection". It asynchronously collects this ID to
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 976d53a870e..36aeb14c563 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -270,6 +273,8 @@ $node_A->psql('postgres',
###############################################################################
# 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.
+# Also, check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -288,6 +293,8 @@ $node_A->poll_query_until('postgres',
"SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
);
+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;");
@@ -299,10 +306,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
@@ -324,6 +351,41 @@ ok( $stderr =~
qr/1 removed, 1 remain, 0 are dead but not yet removable/,
'the deleted column is removed');
+###############################################################################
+# Ensure that the deleted tuple needed to detect an update_deleted conflict is
+# accessible via a sequential table scan.
+###############################################################################
+
+# Drop the primary key from tab on node A and set REPLICA IDENTITY to FULL to
+# enforce sequential scanning of the table.
+$node_A->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_B->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_A->safe_psql('postgres', "ALTER TABLE tab DROP CONSTRAINT tab_pkey;");
+
+# Disable the logical replication from node B to node A
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE");
+
+# Wait for the apply worker to stop
+$node_A->poll_query_until('postgres',
+ "SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
+);
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 4 WHERE a = 2;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 2;");
+
+$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 \(2, 4\); replica identity full \(2, 2\)/,
+ 'update target row was deleted in tab');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.31.1
v55-0002-Collect-statistics-for-update_deleted-conflicts.patchapplication/octet-stream; name=v55-0002-Collect-statistics-for-update_deleted-conflicts.patchDownload
From 14534e9c8b915deb9379a8b341d3de7444f0f7dd Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 29 Jul 2025 12:19:37 +0800
Subject: [PATCH v55 2/4] Collect statistics for update_deleted conflicts
---
doc/src/sgml/monitoring.sgml | 11 +++++++++++
src/backend/catalog/system_views.sql | 1 +
src/backend/utils/adt/pgstatfuncs.c | 14 ++++++++------
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 3 ++-
5 files changed, 25 insertions(+), 10 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 823afe1b30b..20f8dbcf541 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2223,6 +2223,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was recently deleted by another
+ source during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_missing</structfield> <type>bigint</type>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f6eca09ee15..77c693f630e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1399,6 +1399,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_insert_exists,
ss.confl_update_origin_differs,
ss.confl_update_exists,
+ ss.confl_update_deleted,
ss.confl_update_missing,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..c756c2bebaa 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};
@@ -2197,15 +2197,17 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 6, "confl_update_exists",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_deleted",
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 3ee8fed7e53..118d6da1ace 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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_origin_differs,confl_update_exists,confl_update_deleted,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/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dce8c672b40..6509fda77a9 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2179,13 +2179,14 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_insert_exists,
ss.confl_update_origin_differs,
ss.confl_update_exists,
+ ss.confl_update_deleted,
ss.confl_update_missing,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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_origin_differs, confl_update_exists, confl_update_deleted, confl_update_missing, confl_delete_origin_differs, confl_delete_missing, confl_multiple_unique_conflicts, stats_reset);
pg_stat_sys_indexes| SELECT relid,
indexrelid,
schemaname,
--
2.50.1.windows.1
v55-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v55-0003-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 01a214bb8799167e50949cee99a65ec3aa9cad1f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v55 3/4] 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_dead_tuples 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_dead_tuples 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_dead_tuples 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_dead_tuples' 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_dead_tuples 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 | 43 ++++++
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 | 120 +++++++++++++++--
src/backend/replication/logical/worker.c | 126 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 348 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..0d6616857e7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_dead_tuples</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 20f8dbcf541..a0c4cb259f6 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 4187191ea74..27e55bb0f35 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -3007,6 +3007,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 77c693f630e..235b6b4f323 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 742d9ba68e9..c0e785b1a8c 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;
@@ -100,9 +103,11 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -465,6 +470,8 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1181,6 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1223,6 +1231,13 @@ ApplyLauncherMain(Datum main_arg)
*/
can_advance_xmin &= sub->enabled;
+ /*
+ * Stop the conflict information retention only if all workers
+ * for subscriptions with retain_dead_tuples enabled have
+ * requested it.
+ */
+ stop_retention &= sub->enabled;
+
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1254,8 +1269,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin,
+ &stop_retention);
/* worker is running already */
continue;
@@ -1265,10 +1281,15 @@ ApplyLauncherMain(Datum main_arg)
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_dead_tuples are
* running, disabling the further computation of the minimum
- * nonremovable xid.
+ * nonremovable xid. Similarly, stop the conflict information
+ * retention only if all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
*/
if (sub->retaindeadtuples)
+ {
can_advance_xmin = false;
+ stop_retention = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1314,13 +1335,16 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain dead tuples. Otherwise, if required,
- * advance the slot's xmin to protect dead tuples required for the
- * conflict detection.
+ * Invalidate the slot if requested by all apply workers. Or, drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain dead tuples. Otherwise, if required, advance
+ * the slot's xmin to protect dead tuples required for the conflict
+ * detection.
*/
if (MyReplicationSlot)
{
+ if (retain_dead_tuples && stop_retention)
+ invalidate_conflict_slot();
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
@@ -1358,11 +1382,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention)
{
TransactionId nonremovable_xid;
+ bool stop_conflict_info_retention;
Assert(worker != NULL);
@@ -1374,8 +1403,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_dead_tuples enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1436,6 +1479,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_dead_tuples enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1516,7 +1590,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;
@@ -1593,6 +1667,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->subretaindeadtuples &&
+ !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 2959657bf3f..36b3446c2f9 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4100,7 +4105,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4315,6 +4321,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4458,6 +4468,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4539,6 +4556,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4584,12 +4622,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4597,9 +4644,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4632,6 +4736,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8605776ad86..6a7d2076385 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"},
};
/*
@@ -1665,6 +1666,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();
}
@@ -1783,6 +1789,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s,
}
}
+ if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION)
+ {
+ if (IsSlotForConflictCheck(NameStr(s->data.name)))
+ return RS_INVAL_CONFLICT_RETENTION_DURATION;
+ }
+
return RS_INVAL_NONE;
}
@@ -2044,6 +2056,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 d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 118d6da1ace..c284b4d2c6d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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 e8fc342d1a9..2f459404c9e 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 7c0204dd6f4..5bcd5c84844 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining conflict
+ * information. This is used only when retain_dead_tuples 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 6509fda77a9..30dafcfe10c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2169,9 +2169,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..aa6ef19dd70 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Thursday, July 31, 2025 5:26 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:This is the V54 patch set, with only patch 0001 updated to address the
latest comments.Thanks for the patch.
While performing tests on the latest patch, I found an assert in tablesync
worker in FindDeletedTupleInLocalRel (see
Assert(TransactionIdIsValid(oldestxmin))). Logs at [1].It seems table sync worker is trying to apply changes and going to
update-deleted conflict detection patch but is not able to find
MyLogicalRepWorker->oldest_nonremovable_xid as this xid is set only
for apply-worker.
Thanks for reporting. I have fixed it by referring to conflict detection slot's
xmin instead of the leader worker's oldest_nonremovable_xid. This should
be safe because the slot.xmin is always valid.
Best Regards,
Hou zj
On Thu, Jul 31, 2025 at 3:49 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thursday, July 31, 2025 5:26 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:This is the V54 patch set, with only patch 0001 updated to address the
latest comments.Thanks for the patch.
While performing tests on the latest patch, I found an assert in tablesync
worker in FindDeletedTupleInLocalRel (see
Assert(TransactionIdIsValid(oldestxmin))). Logs at [1].It seems table sync worker is trying to apply changes and going to
update-deleted conflict detection patch but is not able to find
MyLogicalRepWorker->oldest_nonremovable_xid as this xid is set only
for apply-worker.Thanks for reporting. I have fixed it by referring to conflict detection slot's
xmin instead of the leader worker's oldest_nonremovable_xid. This should
be safe because the slot.xmin is always valid.
Thanks for fixing. In the same context, the comment below still
mentions oldest_nonremovable_xid, it can be corrected.
+ /*
+ * No need to check for a frozen transaction ID, as
+ * TransactionIdPrecedes() manages it internally, treating it as falling
+ * behind the oldest_nonremovable_xid.
+ */
~
Also we may mention 'concurrently deleted' in the comment below as
that makes more sense.
/* The row to be updated was deleted by a different origin */
CT_UPDATE_DELETED,
~
Apart from these trivial comments changes, patch001 and patch002 look
good to me.
thanks
Shveta
On Thu, Jul 31, 2025 at 3:49 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thursday, July 31, 2025 5:29 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:This is the V54 patch set, with only patch 0001 updated to address the
latest comments.Few minor comments:
Thanks for the comments.
1.
/* 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,
/* The updated row value violates unique constraint */ CT_UPDATE_EXISTS,
/* The row to be updated is missing */
CT_UPDATE_MISSING,Is there a reason to keep CT_UPDATE_DELETED before
CT_UPDATE_ORIGIN_DIFFERS? I mean why not keep it just before
CT_UPDATE_MISSING on the grounds that they are always handled together?I agree that it makes more sense to put it before update_missing, and changed it.
2. Will it be better to name FindRecentlyDeletedTupleInfoByIndex as
RelationFindDeletedTupleInfoByIndex to make it similar to existing function
RelationFindReplTupleByIndex? If you agree then make a similar change for
FindRecentlyDeletedTupleInfoSeq as well.Yes, the suggested name looks better.
Apart from above, please find a number of comment edits and other cosmetic
changes in the attached.Thanks, I have addressed above comments and merge the patch into 0001.
I have few comments in 0001
1.
+ /* Select the dead tuple with the most recent commit timestamp */
+ if (TransactionIdGetCommitTsData(xmax, &localts, &localorigin) &&
+ (TimestampDifferenceExceeds(*delete_time, localts, 0) ||
+ *delete_time == 0))
IMHO the "*delete_time == 0" is a redundant check, because if
*delete_time is 0 then TimestampDifferenceExceeds will always be true
as localts can not be 0.
2.
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named
<quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the conflict information from
being removed.
"to prevent the conflict information from being removed." should be rewritten as
"to prevent removal of tuple required for conflict detection"
3.
+ /* Return if the commit timestamp data is not available */
+ if (!track_commit_timestamp)
+ return false;
Shouldn't caller should take care of this? I mean if the 'retaindeadtuples' and
'track_commit_timestamp' is not set then caller shouldn't even call
this function.
4.
+ /*
+ * Instead of invoking GetOldestNonRemovableTransactionId() for conflict
+ * detection, we use the conflict detection slot.xmin. This value will be
+ * greater than or equal to the other threshold and provides a more direct
+ * and efficient way to identify recently deleted dead tuples relevant to
+ * the conflict detection. The oldest_nonremovable_xid is not used here,
+ * as it is maintained only by the leader apply worker and unavailable to
+ * table sync and parallel apply workers.
+ */
+ slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
This comment seems a bit confusing to me, Isn't it actually correct to
just use the "conflict detection slot.xmin" even without any other
reasoning?
--
Regards,
Dilip Kumar
Google
On Fri, Aug 1, 2025 at 3:58 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
4. + /* + * Instead of invoking GetOldestNonRemovableTransactionId() for conflict + * detection, we use the conflict detection slot.xmin. This value will be + * greater than or equal to the other threshold and provides a more direct + * and efficient way to identify recently deleted dead tuples relevant to + * the conflict detection. The oldest_nonremovable_xid is not used here, + * as it is maintained only by the leader apply worker and unavailable to + * table sync and parallel apply workers. + */ + slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);This comment seems a bit confusing to me, Isn't it actually correct to
just use the "conflict detection slot.xmin" even without any other
reasoning?
But it is *not* wrong to use even GetOldestNonRemovableTransactionId()
because it will anyway consider conflict detection slot's xmin.
However, the value returned by that function could be much older, so
slot's xmin is a better choice. Similarly, it is sufficient to use
oldest_nonremovable_xid value of apply worker and ideally would be
better than slot's xmin because it could give update_deleted in fewer
cases, however, we can't use that because of reasons mentioned in the
comments. Do you think this comment needs improvement for clarity and
if so, do you have any proposal?
--
With Regards,
Amit Kapila.
On Fri, Aug 1, 2025 at 4:20 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Aug 1, 2025 at 3:58 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
4. + /* + * Instead of invoking GetOldestNonRemovableTransactionId() for conflict + * detection, we use the conflict detection slot.xmin. This value will be + * greater than or equal to the other threshold and provides a more direct + * and efficient way to identify recently deleted dead tuples relevant to + * the conflict detection. The oldest_nonremovable_xid is not used here, + * as it is maintained only by the leader apply worker and unavailable to + * table sync and parallel apply workers. + */ + slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);This comment seems a bit confusing to me, Isn't it actually correct to
just use the "conflict detection slot.xmin" even without any other
reasoning?But it is *not* wrong to use even GetOldestNonRemovableTransactionId()
because it will anyway consider conflict detection slot's xmin.
However, the value returned by that function could be much older, so
slot's xmin is a better choice. Similarly, it is sufficient to use
oldest_nonremovable_xid value of apply worker and ideally would be
better than slot's xmin because it could give update_deleted in fewer
cases, however, we can't use that because of reasons mentioned in the
comments. Do you think this comment needs improvement for clarity and
if so, do you have any proposal?
How about something like:
/*
* For conflict detection, we use the conflict slot's xmin value instead of
* invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as a
* threshold to identify tuples that were recently deleted. These tuples are
* not visible to concurrent transactions, but we log an update_deleted conflict
* if such a tuple matches the remote update being applied.
*
* Although GetOldestNonRemovableTransactionId() can return a value older than
* the slot's xmin, for our current purpose it is acceptable to treat tuples
* deleted by transactions prior to slot.xmin as update_missing conflicts.
*
* Ideally, we would use oldest_nonremovable_xid, which is directly maintained
* by the leader apply worker. However, this value is not available to table
* synchronization or parallel apply workers, making slot.xmin a practical
* alternative in those contexts.
*/
--
With Regards,
Amit Kapila.
-----Original Message-----
From: Dilip Kumar <dilipbalaut@gmail.com>
Sent: Friday, August 1, 2025 6:29 PM
To: Hou, Zhijie/侯 志杰 <houzj.fnst@fujitsu.com>
Cc: Amit Kapila <amit.kapila16@gmail.com>; shveta malik
<shveta.malik@gmail.com>; Kuroda, Hayato/黒田 隼人
<kuroda.hayato@fujitsu.com>; pgsql-hackers
<pgsql-hackers@postgresql.org>; vignesh C <vignesh21@gmail.com>; Nisha
Moond <nisha.moond412@gmail.com>; Masahiko Sawada
<sawada.mshk@gmail.com>
Subject: Re: Conflict detection for update_deleted in logical replicationOn Thu, Jul 31, 2025 at 3:49 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:On Thursday, July 31, 2025 5:29 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
On Tue, Jul 29, 2025 at 10:51 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:This is the V54 patch set, with only patch 0001 updated to address
the latest comments.Few minor comments:
Thanks for the comments.
1.
/* 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,
/* The updated row value violates unique constraint */
CT_UPDATE_EXISTS,
/* The row to be updated is missing */ CT_UPDATE_MISSING,Is there a reason to keep CT_UPDATE_DELETED before
CT_UPDATE_ORIGIN_DIFFERS? I mean why not keep it just before
CT_UPDATE_MISSING on the grounds that they are always handledtogether?
I agree that it makes more sense to put it before update_missing, and
changed it.
2. Will it be better to name FindRecentlyDeletedTupleInfoByIndex as
RelationFindDeletedTupleInfoByIndex to make it similar to existing
function RelationFindReplTupleByIndex? If you agree then make a
similar change for FindRecentlyDeletedTupleInfoSeq as well.Yes, the suggested name looks better.
Apart from above, please find a number of comment edits and other
cosmetic changes in the attached.Thanks, I have addressed above comments and merge the patch into 0001.
I have few comments in 0001
Thanks for the comments!
2.
+ If set to <literal>true</literal>, the detection of + <xref linkend="conflict-update-deleted"/> is enabled, and a physical + replication slot named <quote><literal>pg_conflict_detection</literal></quote> created on the subscriber to prevent the conflict information from being removed."to prevent the conflict information from being removed." should be rewritten
as "to prevent removal of tuple required for conflict detection"
It appears the document you commented is already committed. I think the
intention was to make a general statement that neither dead tuples nor commit
timestamp data would be removed.
3. + /* Return if the commit timestamp data is not available */ + if (!track_commit_timestamp) + return false;Shouldn't caller should take care of this? I mean if the 'retaindeadtuples' and
'track_commit_timestamp' is not set then caller shouldn't even call this
function.
I feel moving the checks into a single central function would streamline the
caller, reducing code duplication. So, maybe we could move the retaindeadtuple
check into this function as well for consistency. Thoughts ?
Best Regards,
Hou zj
On Fri, Aug 1, 2025 at 4:46 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Aug 1, 2025 at 4:20 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Aug 1, 2025 at 3:58 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
4. + /* + * Instead of invoking GetOldestNonRemovableTransactionId() for conflict + * detection, we use the conflict detection slot.xmin. This value will be + * greater than or equal to the other threshold and provides a more direct + * and efficient way to identify recently deleted dead tuples relevant to + * the conflict detection. The oldest_nonremovable_xid is not used here, + * as it is maintained only by the leader apply worker and unavailable to + * table sync and parallel apply workers. + */ + slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);This comment seems a bit confusing to me, Isn't it actually correct to
just use the "conflict detection slot.xmin" even without any other
reasoning?But it is *not* wrong to use even GetOldestNonRemovableTransactionId()
because it will anyway consider conflict detection slot's xmin.
However, the value returned by that function could be much older, so
slot's xmin is a better choice. Similarly, it is sufficient to use
oldest_nonremovable_xid value of apply worker and ideally would be
better than slot's xmin because it could give update_deleted in fewer
cases, however, we can't use that because of reasons mentioned in the
comments.
Got it. It Makes sense to give other possibility and why we chose slot.xmin
How about something like:
/*
* For conflict detection, we use the conflict slot's xmin value instead of
* invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as a
* threshold to identify tuples that were recently deleted. These tuples are
* not visible to concurrent transactions, but we log an update_deleted conflict
* if such a tuple matches the remote update being applied.
*
* Although GetOldestNonRemovableTransactionId() can return a value older than
* the slot's xmin, for our current purpose it is acceptable to treat tuples
* deleted by transactions prior to slot.xmin as update_missing conflicts.
*
* Ideally, we would use oldest_nonremovable_xid, which is directly maintained
* by the leader apply worker. However, this value is not available to table
* synchronization or parallel apply workers, making slot.xmin a practical
* alternative in those contexts.
*/
I think this is much better.
--
Regards,
Dilip Kumar
Google
On Fri, Aug 1, 2025 at 5:02 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
2.
+ If set to <literal>true</literal>, the detection of + <xref linkend="conflict-update-deleted"/> is enabled, and a physical + replication slot named <quote><literal>pg_conflict_detection</literal></quote> created on the subscriber to prevent the conflict information from being removed."to prevent the conflict information from being removed." should be rewritten
as "to prevent removal of tuple required for conflict detection"It appears the document you commented is already committed. I think the
intention was to make a general statement that neither dead tuples nor commit
timestamp data would be removed.
Okay got it, so instead of "conflict information" should we say
"information for detecting conflicts" or "conflict detection
information", conflict information looks like we want to prevent the
information about the conflict which has already happened, instead we
are preventing information which are required for detecting the
conflict, does this make sense?
I know this is already committed, but actually this is part of the
whole patch set so we can always improvise it.
3. + /* Return if the commit timestamp data is not available */ + if (!track_commit_timestamp) + return false;Shouldn't caller should take care of this? I mean if the 'retaindeadtuples' and
'track_commit_timestamp' is not set then caller shouldn't even call this
function.I feel moving the checks into a single central function would streamline the
caller, reducing code duplication. So, maybe we could move the retaindeadtuple
check into this function as well for consistency. Thoughts ?
Fine with either way, actually I wanted both the check
'retaindeadtuple' and 'track_commit_timestamp' at the same place.
--
Regards,
Dilip Kumar
Google
On Friday, August 1, 2025 7:42 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Aug 1, 2025 at 5:02 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:2.
+ If set to <literal>true</literal>, the detection of + <xref linkend="conflict-update-deleted"/> is enabled, and + a physical + replication slot named <quote><literal>pg_conflict_detection</literal></quote> created on the subscriber to prevent the conflict informationfrom
being removed.
"to prevent the conflict information from being removed." should be
rewritten as "to prevent removal of tuple required for conflict detection"It appears the document you commented is already committed. I think
the intention was to make a general statement that neither dead tuples
nor commit timestamp data would be removed.Okay got it, so instead of "conflict information" should we say "information for
detecting conflicts" or "conflict detection information", conflict information
looks like we want to prevent the information about the conflict which has
already happened, instead we are preventing information which are required
for detecting the conflict, does this make sense?
It makes sense to me, so changed.
I know this is already committed, but actually this is part of the whole patch set
so we can always improvise it.3. + /* Return if the commit timestamp data is not available */ + if (!track_commit_timestamp) + return false;Shouldn't caller should take care of this? I mean if the
'retaindeadtuples' and 'track_commit_timestamp' is not set then
caller shouldn't even call this function.I feel moving the checks into a single central function would
streamline the caller, reducing code duplication. So, maybe we could
move the retaindeadtuple check into this function as well for consistency.Thoughts ?
Fine with either way, actually I wanted both the check 'retaindeadtuple' and
'track_commit_timestamp' at the same place.
Thanks for confirming. Here is V56 patch set which addressed all the
comments including the comments from Amit[1]/messages/by-id/CAA4eK1+2tZ0rGowwpfmPQA03KdBOaeaK6D5omBN76UTP2EPx6w@mail.gmail.com and Shveta[2]/messages/by-id/CAJpy0uDNhP+QeH-zGLBgMnRY1JZGVeoZ_dxff5S6HmpnRcWk8A@mail.gmail.com.
I have merged V55-0002 into 0001 and updated the list of author
and reviewers based on my knowledge.
[1]: /messages/by-id/CAA4eK1+2tZ0rGowwpfmPQA03KdBOaeaK6D5omBN76UTP2EPx6w@mail.gmail.com
[2]: /messages/by-id/CAJpy0uDNhP+QeH-zGLBgMnRY1JZGVeoZ_dxff5S6HmpnRcWk8A@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v56-0003-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v56-0003-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 0abb5c4a633746fc5edbf020c3413f781394ec06 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:41:37 +0800
Subject: [PATCH v56 3/3] 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 | 15 ++++-
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
src/backend/utils/adt/pg_upgrade_support.c | 2 +-
src/include/replication/logicallauncher.h | 2 +-
5 files changed, 63 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 0d6616857e7..e616381e44a 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5425,7 +5425,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_dead_tuples</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0e785b1a8c..50d52cbe77f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1252,7 +1252,7 @@ ApplyLauncherMain(Datum main_arg)
* applying remote changes that occurred before the
* subscription was enabled.
*/
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(!sub->enabled);
}
if (!sub->enabled)
@@ -1289,6 +1289,12 @@ ApplyLauncherMain(Datum main_arg)
{
can_advance_xmin = false;
stop_retention = false;
+
+ /*
+ * Re-create the slot if it has been invalidated, as retention
+ * has now resumed.
+ */
+ CreateConflictDetectionSlot(true);
}
/*
@@ -1515,10 +1521,15 @@ invalidate_conflict_slot(void)
* conflict detection, if not yet.
*/
void
-CreateConflictDetectionSlot(void)
+CreateConflictDetectionSlot(bool recreate_if_invalid)
{
TransactionId xmin_horizon;
+ /* Drop the invalidated slot and re-create it if requested */
+ if (recreate_if_invalid && MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d8d929c252b..fce2c9e9612 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -592,6 +592,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4321,10 +4323,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4606,6 +4604,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4613,6 +4630,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4655,9 +4673,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_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_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4687,19 +4704,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index a4f8b4faa90..e20fc44adda 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -423,7 +423,7 @@ binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
{
CHECK_IS_BINARY_UPGRADE;
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(false);
ReplicationSlotRelease();
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 6e3007db5f0..5052c394c8f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -29,7 +29,7 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
-extern void CreateConflictDetectionSlot(void);
+extern void CreateConflictDetectionSlot(bool recreate_if_invalid);
extern bool IsLogicalLauncher(void);
--
2.50.1.windows.1
v56-0001-Support-the-conflict-detection-for-update_delete.patchapplication/octet-stream; name=v56-0001-Support-the-conflict-detection-for-update_delete.patchDownload
From 678c52ec9edee062543bea77146bac3ec4a2a960 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 29 Jul 2025 12:18:24 +0800
Subject: [PATCH v56] 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.
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
Reviewed-by: Hayato Kuroda <kuroda.hayato@fujitsu.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
---
doc/src/sgml/catalogs.sgml | 3 +-
doc/src/sgml/logical-replication.sgml | 16 ++
doc/src/sgml/monitoring.sgml | 11 +
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/catalog/system_views.sql | 1 +
src/backend/executor/execReplication.c | 251 ++++++++++++++++++++-
src/backend/replication/logical/conflict.c | 22 ++
src/backend/replication/logical/worker.c | 186 +++++++++++++--
src/backend/utils/adt/pgstatfuncs.c | 14 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/executor/executor.h | 14 +-
src/include/replication/conflict.h | 3 +
src/include/replication/worker_internal.h | 5 +-
src/test/regress/expected/rules.out | 3 +-
src/test/subscription/t/035_conflicts.pl | 66 +++++-
15 files changed, 565 insertions(+), 45 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index 97f547b3cc4..da8a7882580 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8087,7 +8087,8 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
<structfield>subretaindeadtuples</structfield> <type>bool</type>
</para>
<para>
- If true, the information (e.g., dead tuples, commit timestamps, and
+ If true, the detection of <xref linkend="conflict-update-deleted"/> is
+ enabled and the information (e.g., dead tuples, commit timestamps, and
origins) on the subscriber that is useful for conflict detection is
retained.
</para></entry>
diff --git a/doc/src/sgml/logical-replication.sgml b/doc/src/sgml/logical-replication.sgml
index fcac55aefe6..a0761cfee3f 100644
--- a/doc/src/sgml/logical-replication.sgml
+++ b/doc/src/sgml/logical-replication.sgml
@@ -1804,6 +1804,22 @@ Publications:
</para>
</listitem>
</varlistentry>
+ <varlistentry id="conflict-update-deleted" xreflabel="update_deleted">
+ <term><literal>update_deleted</literal></term>
+ <listitem>
+ <para>
+ The tuple to be updated was concurrently deleted by another origin. The
+ update will simply be skipped in this scenario. Note that this conflict
+ can only be detected when
+ <link linkend="guc-track-commit-timestamp"><varname>track_commit_timestamp</varname></link>
+ and <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ are enabled. Note that if a tuple cannot be found due to the table being
+ truncated, only a <literal>update_missing</literal> conflict will
+ arise. Additionally, if the tuple was deleted by the same origin, an
+ <literal>update_missing</literal> conflict will arise.
+ </para>
+ </listitem>
+ </varlistentry>
<varlistentry id="conflict-update-missing" xreflabel="update_missing">
<term><literal>update_missing</literal></term>
<listitem>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 823afe1b30b..fa78031ccbb 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2223,6 +2223,17 @@ description | Waiting for a newly initialized WAL file to reach durable storage
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>confl_update_deleted</structfield> <type>bigint</type>
+ </para>
+ <para>
+ Number of times the tuple to be updated was concurrently deleted by
+ another source during the application of changes. See <xref linkend="conflict-update-deleted"/>
+ for details about this conflict.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>confl_update_missing</structfield> <type>bigint</type>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index b8cd15f3280..247c5bd2604 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -445,10 +445,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
Specifies whether the information (e.g., dead tuples, commit
timestamps, and origins) required for conflict detection on the
subscriber is retained. The default is <literal>false</literal>.
- If set to <literal>true</literal>, a physical replication slot named
- <quote><literal>pg_conflict_detection</literal></quote> will be
- created on the subscriber to prevent the conflict information from
- being removed.
+ If set to <literal>true</literal>, the detection of
+ <xref linkend="conflict-update-deleted"/> is enabled, and a physical
+ replication slot named <quote><literal>pg_conflict_detection</literal></quote>
+ created on the subscriber to prevent the information for detecting
+ conflicts from being removed.
</para>
<para>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f6eca09ee15..77c693f630e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1399,6 +1399,7 @@ CREATE VIEW pg_stat_subscription_stats AS
ss.confl_insert_exists,
ss.confl_update_origin_differs,
ss.confl_update_exists,
+ ss.confl_update_deleted,
ss.confl_update_missing,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
diff --git a/src/backend/executor/execReplication.c b/src/backend/executor/execReplication.c
index f262e7a66f7..68184f5d671 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"
@@ -36,7 +38,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 +223,7 @@ retry:
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 +279,13 @@ retry:
/*
* 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 +310,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 +393,7 @@ retry:
/* 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 +468,236 @@ BuildConflictIndexInfo(ResultRelInfo *resultRelInfo, Oid conflictindex)
}
}
+/*
+ * If the tuple is recently dead and was deleted by a transaction with a newer
+ * commit timestamp than previously recorded, update the associated transaction
+ * ID, commit time, and origin. This helps ensure that conflict detection uses
+ * the most recent and relevant deletion metadata.
+ */
+static void
+update_most_recent_deletion_info(TupleTableSlot *scanslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ TimestampTz *delete_time,
+ RepOriginId *delete_origin)
+{
+ BufferHeapTupleTableSlot *hslot;
+ HeapTuple tuple;
+ Buffer buf;
+ bool recently_dead = false;
+ TransactionId xmax;
+ TimestampTz localts;
+ RepOriginId localorigin;
+
+ hslot = (BufferHeapTupleTableSlot *) scanslot;
+
+ 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 worker.c for details).
+ */
+ if (HeapTupleSatisfiesVacuum(tuple, oldestxmin, buf) == HEAPTUPLE_RECENTLY_DEAD)
+ recently_dead = true;
+
+ LockBuffer(buf, BUFFER_LOCK_UNLOCK);
+
+ if (!recently_dead)
+ return;
+
+ xmax = HeapTupleHeaderGetUpdateXid(tuple->t_data);
+ if (!TransactionIdIsValid(xmax))
+ return;
+
+ /* Select the dead tuple with the most recent commit timestamp */
+ if (TransactionIdGetCommitTsData(xmax, &localts, &localorigin) &&
+ TimestampDifferenceExceeds(*delete_time, localts, 0))
+ {
+ *delete_xid = xmax;
+ *delete_time = localts;
+ *delete_origin = localorigin;
+ }
+}
+
+/*
+ * 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.
+ *
+ * 'oldestxmin' acts as a cutoff transaction ID. Tuples deleted by transactions
+ * with IDs >= 'oldestxmin' are considered recently dead and are eligible for
+ * conflict detection.
+ *
+ * Instead of stopping at the first match, we scan all matching dead tuples to
+ * identify most recent deletion. This is crucial because only the latest
+ * deletion is relevant for resolving conflicts.
+ *
+ * For example, consider a scenario on the subscriber where a row is deleted,
+ * re-inserted, and then deleted again only on the subscriber:
+ *
+ * - (pk, 1) - deleted at 9:00,
+ * - (pk, 1) - deleted at 9:02,
+ *
+ * Now, a remote update arrives: (pk, 1) -> (pk, 2), timestamped at 9:01.
+ *
+ * If we mistakenly return the older deletion (9:00), the system may wrongly
+ * apply the remote update using a last-update-wins strategy. Instead, we must
+ * recognize the more recent deletion at 9:02 and skip the update. See
+ * comments atop worker.c for details. Note, as of now, conflict resolution
+ * is not implemented. Consequently, the system may incorrectly report the
+ * older tuple as the conflicted one, leading to misleading results.
+ *
+ * The commit timestamp of the deleting transaction is used to determine which
+ * tuple was deleted most recently.
+ */
+bool
+RelationFindDeletedTupleInfoSeq(Relation rel, TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TupleTableSlot *scanslot;
+ TableScanDesc scan;
+ TypeCacheEntry **eq;
+ 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;
+
+ /*
+ * If the relation has a replica identity key or a primary key that is
+ * unusable for locating deleted tuples (see
+ * IsIndexUsableForFindingDeletedTuple), a full table scan becomes
+ * necessary. In such cases, comparing the entire tuple is not required,
+ * since the remote tuple might not include all column values. Instead,
+ * the indexed columns alone are suffcient to identify the target tuple
+ * (see logicalrep_rel_mark_updatable).
+ */
+ 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. Tuples from transactions
+ * not yet committed or those just committed prior to the scan are
+ * excluded in update_most_recent_deletion_info().
+ */
+ scan = table_beginscan(rel, SnapshotAny, 0, NULL);
+ scanslot = table_slot_create(rel, NULL);
+
+ table_rescan(scan, NULL);
+
+ /* Try to find the tuple */
+ while (table_scan_getnextslot(scan, ForwardScanDirection, scanslot))
+ {
+ if (!tuples_equal(scanslot, searchslot, eq, indexbitmap))
+ continue;
+
+ update_most_recent_deletion_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ table_endscan(scan);
+ ExecDropSingleTupleTableSlot(scanslot);
+
+ return *delete_time != 0;
+}
+
+/*
+ * Similar to RelationFindDeletedTupleInfoSeq() but using index scan to locate
+ * the deleted tuple.
+ */
+bool
+RelationFindDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ Relation idxrel;
+ ScanKeyData skey[INDEX_MAX_KEYS];
+ int skey_attoff;
+ IndexScanDesc scan;
+ TupleTableSlot *scanslot;
+ TypeCacheEntry **eq = NULL;
+ bool isIdxSafeToSkipDuplicates;
+ TupleDesc desc PG_USED_FOR_ASSERTS_ONLY = RelationGetDescr(rel);
+
+ Assert(equalTupleDescs(desc, searchslot->tts_tupleDescriptor));
+ Assert(OidIsValid(idxoid));
+
+ *delete_xid = InvalidTransactionId;
+ *delete_time = 0;
+ *delete_origin = InvalidRepOriginId;
+
+ isIdxSafeToSkipDuplicates = (GetRelationIdentityOrPK(rel) == idxoid);
+
+ scanslot = table_slot_create(rel, NULL);
+
+ idxrel = index_open(idxoid, RowExclusiveLock);
+
+ /* Build scan key. */
+ skey_attoff = build_replindex_scan_key(skey, rel, idxrel, searchslot);
+
+ /*
+ * Start an index scan using SnapshotAny to identify dead tuples that are
+ * not visible under a standard MVCC snapshot. Tuples from transactions
+ * not yet committed or those just committed prior to the scan are
+ * excluded in update_most_recent_deletion_info().
+ */
+ scan = index_beginscan(rel, idxrel, SnapshotAny, NULL, skey_attoff, 0);
+
+ index_rescan(scan, skey, skey_attoff, NULL, 0);
+
+ /* Try to find the tuple */
+ while (index_getnext_slot(scan, ForwardScanDirection, scanslot))
+ {
+ /*
+ * Avoid expensive equality check if the index is primary key or
+ * replica identity index.
+ */
+ if (!isIdxSafeToSkipDuplicates)
+ {
+ if (eq == NULL)
+ eq = palloc0(sizeof(*eq) * scanslot->tts_tupleDescriptor->natts);
+
+ if (!tuples_equal(scanslot, searchslot, eq, NULL))
+ continue;
+ }
+
+ update_most_recent_deletion_info(scanslot, oldestxmin, delete_xid,
+ delete_time, delete_origin);
+ }
+
+ index_endscan(scan);
+
+ index_close(idxrel, NoLock);
+
+ 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 97c4e26b586..2fd3e8bbda5 100644
--- a/src/backend/replication/logical/conflict.c
+++ b/src/backend/replication/logical/conflict.c
@@ -29,6 +29,7 @@ static const char *const ConflictTypeNames[] = {
[CT_UPDATE_EXISTS] = "update_exists",
[CT_UPDATE_MISSING] = "update_missing",
[CT_DELETE_ORIGIN_DIFFERS] = "delete_origin_differs",
+ [CT_UPDATE_DELETED] = "update_deleted",
[CT_DELETE_MISSING] = "delete_missing",
[CT_MULTIPLE_UNIQUE_CONFLICTS] = "multiple_unique_conflicts"
};
@@ -176,6 +177,7 @@ errcode_apply_conflict(ConflictType type)
case CT_UPDATE_ORIGIN_DIFFERS:
case CT_UPDATE_MISSING:
case CT_DELETE_ORIGIN_DIFFERS:
+ case CT_UPDATE_DELETED:
case CT_DELETE_MISSING:
return errcode(ERRCODE_T_R_SERIALIZATION_FAILURE);
}
@@ -261,6 +263,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_MISSING:
appendStringInfoString(&err_detail, _("Could not find the row to be updated."));
break;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b59221c4d06..89e241c8392 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -138,9 +138,9 @@
* Each apply worker that enabled retain_dead_tuples option maintains a
* non-removable transaction ID (oldest_nonremovable_xid) in shared memory to
* prevent dead rows from being removed prematurely when the apply worker still
- * needs them to detect conflicts reliably. This helps to retain the required
- * commit_ts module information, which further helps to detect
- * update_origin_differs and delete_origin_differs conflicts reliably, as
+ * needs them to detect update_deleted conflicts. Additionally, this helps to
+ * retain the required commit_ts module information, which further helps to
+ * detect update_origin_differs and delete_origin_differs conflicts reliably, as
* otherwise, vacuum freeze could remove the required information.
*
* The logical replication launcher manages an internal replication slot named
@@ -185,10 +185,10 @@
* 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, detecting concurrent remote
- * transactions with earlier timestamps than the DELETE is necessary, as the
- * UPDATEs in remote transactions should be ignored if their timestamp is
- * earlier than that of the dead tuples.
+ * 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
@@ -576,6 +576,12 @@ static bool FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel
Oid localidxoid,
TupleTableSlot *remoteslot,
TupleTableSlot **localslot);
+static bool FindDeletedTupleInLocalRel(Relation localrel,
+ Oid localidxoid,
+ TupleTableSlot *remoteslot,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
static void apply_handle_tuple_routing(ApplyExecutionData *edata,
TupleTableSlot *remoteslot,
LogicalRepTupleData *newtup,
@@ -2912,17 +2918,31 @@ apply_handle_update_internal(ApplyExecutionData *edata,
}
else
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ /*
+ * Detecting whether the tuple was recently deleted or never existed
+ * is crucial to avoid misleading the user during confict handling.
+ */
+ if (FindDeletedTupleInLocalRel(localrel, localindexoid, 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. */
@@ -3142,6 +3162,112 @@ FindReplTupleInLocalRel(ApplyExecutionData *edata, Relation localrel,
return found;
}
+/*
+ * Determine whether the index can reliably locate the deleted tuple in the
+ * local relation.
+ *
+ * An index may exclude deleted tuples if it was re-indexed or re-created during
+ * change application. Therefore, an index is considered usable only if the
+ * conflict detection slot.xmin (conflict_detection_xmin) is greater than the
+ * index tuple's xmin. This ensures that any tuples deleted prior to the index
+ * creation or re-indexing are not relevant for conflict detection in the
+ * current apply worker.
+ *
+ * Note that indexes may also be excluded if they were modified by other DDL
+ * operations, such as ALTER INDEX. However, this is acceptable, as the
+ * likelihood of such DDL changes coinciding with the need to scan dead
+ * tuples for the update_deleted is low.
+ */
+static bool
+IsIndexUsableForFindingDeletedTuple(Oid localindexoid,
+ TransactionId conflict_detection_xmin)
+{
+ HeapTuple index_tuple;
+ TransactionId index_xmin;
+
+ index_tuple = SearchSysCache1(INDEXRELID, ObjectIdGetDatum(localindexoid));
+
+ if (!HeapTupleIsValid(index_tuple)) /* should not happen */
+ elog(ERROR, "cache lookup failed for index %u", localindexoid);
+
+ /*
+ * No need to check for a frozen transaction ID, as
+ * TransactionIdPrecedes() manages it internally, treating it as falling
+ * behind the conflict_detection_xmin.
+ */
+ index_xmin = HeapTupleHeaderGetXmin(index_tuple->t_data);
+
+ ReleaseSysCache(index_tuple);
+
+ return TransactionIdPrecedes(index_xmin, conflict_detection_xmin);
+}
+
+/*
+ * Attempts to locate a deleted tuple in the local relation that matches the
+ * values of the tuple received from the publication side (in 'remoteslot').
+ * The search is performed using either the replica identity index, primary
+ * key, other available index, or a sequential scan if necessary.
+ *
+ * Returns true if the deleted tuple is found. If found, the transaction ID,
+ * origin, and commit timestamp of the deletion are stored in '*delete_xid',
+ * '*delete_origin', and '*delete_time' respectively.
+ */
+static bool
+FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
+ TupleTableSlot *remoteslot,
+ TransactionId *delete_xid, RepOriginId *delete_origin,
+ TimestampTz *delete_time)
+{
+ TransactionId oldestxmin;
+ ReplicationSlot *slot;
+
+ /*
+ * Return false if either dead tuples are not retained or commit timestamp
+ * data is not available.
+ */
+ if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
+ return false;
+
+ /*
+ * For conflict detection, we use the conflict slot's xmin value instead
+ * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
+ * a threshold to identify tuples that were recently deleted. These tuples
+ * are not visible to concurrent transactions, but we log an
+ * update_deleted conflict if such a tuple matches the remote update being
+ * applied.
+ *
+ * Although GetOldestNonRemovableTransactionId() can return a value older
+ * than the slot's xmin, for our current purpose it is acceptable to treat
+ * tuples deleted by transactions prior to slot.xmin as update_missing
+ * conflicts.
+ *
+ * Ideally, we would use oldest_nonremovable_xid, which is directly
+ * maintained by the leader apply worker. However, this value is not
+ * available to table synchronization or parallel apply workers, making
+ * slot.xmin a practical alternative in those contexts.
+ */
+ slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+
+ Assert(slot);
+
+ SpinLockAcquire(&slot->mutex);
+ oldestxmin = slot->data.xmin;
+ SpinLockRelease(&slot->mutex);
+
+ Assert(TransactionIdIsValid(oldestxmin));
+
+ if (OidIsValid(localidxoid) &&
+ IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
+ return RelationFindDeletedTupleInfoByIndex(localrel, localidxoid,
+ remoteslot, oldestxmin,
+ delete_xid, delete_origin,
+ delete_time);
+ else
+ return RelationFindDeletedTupleInfoSeq(localrel, remoteslot,
+ oldestxmin, delete_xid,
+ delete_origin, delete_time);
+}
+
/*
* This handles insert, update, delete on a partitioned table.
*/
@@ -3260,18 +3386,35 @@ apply_handle_tuple_routing(ApplyExecutionData *edata,
remoteslot_part, &localslot);
if (!found)
{
+ ConflictType type;
TupleTableSlot *newslot = localslot;
+ /*
+ * Detecting whether the tuple was recently deleted or
+ * never existed is crucial to avoid misleading the user
+ * during confict handling.
+ */
+ if (FindDeletedTupleInLocalRel(partrel,
+ part_entry->localindexoid,
+ 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;
}
@@ -4172,8 +4315,8 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
{
/*
* It is sufficient to manage non-removable transaction ID for a
- * subscription by the main apply worker to detect conflicts reliably even
- * for table sync or parallel apply workers.
+ * subscription by the main apply worker to detect update_deleted reliably
+ * even for table sync or parallel apply workers.
*/
if (!am_leader_apply_worker())
return false;
@@ -4374,10 +4517,11 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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 conflicts reliably. This check primarily addresses
- * scenarios where the publisher's clock falls behind; if the publisher's
- * clock is ahead, subsequent transactions will naturally bear later
- * commit timestamps, conforming to the design outlined atop worker.c.
+ * required to detect update_deleted reliably. This check primarily
+ * addresses scenarios where the publisher's clock falls behind; if the
+ * publisher's clock is ahead, subsequent transactions will naturally bear
+ * later commit timestamps, conforming to the design outlined atop
+ * worker.c.
*
* XXX Consider waiting for the publisher's clock to catch up with the
* subscriber's before proceeding to the next phase.
diff --git a/src/backend/utils/adt/pgstatfuncs.c b/src/backend/utils/adt/pgstatfuncs.c
index 1c12ddbae49..c756c2bebaa 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};
@@ -2197,15 +2197,17 @@ pg_stat_get_subscription_stats(PG_FUNCTION_ARGS)
INT8OID, -1, 0);
TupleDescInitEntry(tupdesc, (AttrNumber) 6, "confl_update_exists",
INT8OID, -1, 0);
- TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_missing",
+ TupleDescInitEntry(tupdesc, (AttrNumber) 7, "confl_update_deleted",
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 3ee8fed7e53..118d6da1ace 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5688,9 +5688,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_origin_differs,confl_update_exists,confl_update_deleted,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 104b059544d..a71502efeed 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"
@@ -759,7 +760,18 @@ extern bool RelationFindReplTupleByIndex(Relation rel, Oid idxoid,
TupleTableSlot *outslot);
extern bool RelationFindReplTupleSeq(Relation rel, LockTupleMode lockmode,
TupleTableSlot *searchslot, TupleTableSlot *outslot);
-
+extern bool RelationFindDeletedTupleInfoSeq(Relation rel,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ TransactionId *delete_xid,
+ RepOriginId *delete_origin,
+ TimestampTz *delete_time);
+extern bool RelationFindDeletedTupleInfoByIndex(Relation rel, Oid idxoid,
+ TupleTableSlot *searchslot,
+ TransactionId oldestxmin,
+ 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 6c59125f256..ff3cb8416ec 100644
--- a/src/include/replication/conflict.h
+++ b/src/include/replication/conflict.h
@@ -32,6 +32,9 @@ typedef enum
/* The updated row value violates unique constraint */
CT_UPDATE_EXISTS,
+ /* The row to be updated was concurrently deleted by a different origin */
+ CT_UPDATE_DELETED,
+
/* The row to be updated is missing */
CT_UPDATE_MISSING,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 0c7b8440a61..7c0204dd6f4 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -87,8 +87,9 @@ typedef struct LogicalRepWorker
bool parallel_apply;
/*
- * The changes made by this and later transactions must be retained to
- * ensure reliable conflict detection during the apply phase.
+ * Changes made by this transaction and subsequent ones must be preserved.
+ * This ensures that update_deleted conflicts can be accurately detected
+ * during the apply phase of logical replication by this worker.
*
* The logical replication launcher manages an internal replication slot
* named "pg_conflict_detection". It asynchronously collects this ID to
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index dce8c672b40..6509fda77a9 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2179,13 +2179,14 @@ pg_stat_subscription_stats| SELECT ss.subid,
ss.confl_insert_exists,
ss.confl_update_origin_differs,
ss.confl_update_exists,
+ ss.confl_update_deleted,
ss.confl_update_missing,
ss.confl_delete_origin_differs,
ss.confl_delete_missing,
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_origin_differs, confl_update_exists, confl_update_deleted, 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 976d53a870e..36aeb14c563 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -150,7 +150,9 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
# 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
@@ -158,7 +160,8 @@ pass('multiple_unique_conflicts detected on a leaf partition during insert');
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;
@@ -270,6 +273,8 @@ $node_A->psql('postgres',
###############################################################################
# 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.
+# Also, check that an update_deleted conflict is detected when updating a row
+# that was deleted by a different origin.
###############################################################################
# Insert a record
@@ -288,6 +293,8 @@ $node_A->poll_query_until('postgres',
"SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
);
+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;");
@@ -299,10 +306,30 @@ ok( $stderr =~
qr/1 are dead but not yet removable/,
'the deleted column is non-removable');
+# Ensure the DELETE is replayed on Node B
+$node_A->wait_for_catchup($subname_BA);
+
+# Check the conflict detected on Node B
+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;");
@@ -324,6 +351,41 @@ ok( $stderr =~
qr/1 removed, 1 remain, 0 are dead but not yet removable/,
'the deleted column is removed');
+###############################################################################
+# Ensure that the deleted tuple needed to detect an update_deleted conflict is
+# accessible via a sequential table scan.
+###############################################################################
+
+# Drop the primary key from tab on node A and set REPLICA IDENTITY to FULL to
+# enforce sequential scanning of the table.
+$node_A->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_B->safe_psql('postgres', "ALTER TABLE tab REPLICA IDENTITY FULL");
+$node_A->safe_psql('postgres', "ALTER TABLE tab DROP CONSTRAINT tab_pkey;");
+
+# Disable the logical replication from node B to node A
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE");
+
+# Wait for the apply worker to stop
+$node_A->poll_query_until('postgres',
+ "SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
+);
+
+$node_B->safe_psql('postgres', "UPDATE tab SET b = 4 WHERE a = 2;");
+$node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 2;");
+
+$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 \(2, 4\); replica identity full \(2, 2\)/,
+ 'update target row was deleted in tab');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.50.1.windows.1
v56-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v56-0002-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 7bdfd7c8c97b87e23d3d1d9cbf80a901eb9e96c3 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v56 2/3] 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_dead_tuples 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_dead_tuples 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_dead_tuples 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_dead_tuples' 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_dead_tuples 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 | 43 ++++++
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 | 120 +++++++++++++++--
src/backend/replication/logical/worker.c | 126 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 348 insertions(+), 27 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..0d6616857e7 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_dead_tuples</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fa78031ccbb..9a0fd556c5f 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 4187191ea74..27e55bb0f35 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -3007,6 +3007,17 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 77c693f630e..235b6b4f323 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 742d9ba68e9..c0e785b1a8c 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;
@@ -100,9 +103,11 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -465,6 +470,8 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1181,6 +1188,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1223,6 +1231,13 @@ ApplyLauncherMain(Datum main_arg)
*/
can_advance_xmin &= sub->enabled;
+ /*
+ * Stop the conflict information retention only if all workers
+ * for subscriptions with retain_dead_tuples enabled have
+ * requested it.
+ */
+ stop_retention &= sub->enabled;
+
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1254,8 +1269,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin,
+ &stop_retention);
/* worker is running already */
continue;
@@ -1265,10 +1281,15 @@ ApplyLauncherMain(Datum main_arg)
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_dead_tuples are
* running, disabling the further computation of the minimum
- * nonremovable xid.
+ * nonremovable xid. Similarly, stop the conflict information
+ * retention only if all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
*/
if (sub->retaindeadtuples)
+ {
can_advance_xmin = false;
+ stop_retention = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1314,13 +1335,16 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain dead tuples. Otherwise, if required,
- * advance the slot's xmin to protect dead tuples required for the
- * conflict detection.
+ * Invalidate the slot if requested by all apply workers. Or, drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain dead tuples. Otherwise, if required, advance
+ * the slot's xmin to protect dead tuples required for the conflict
+ * detection.
*/
if (MyReplicationSlot)
{
+ if (retain_dead_tuples && stop_retention)
+ invalidate_conflict_slot();
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
@@ -1358,11 +1382,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention)
{
TransactionId nonremovable_xid;
+ bool stop_conflict_info_retention;
Assert(worker != NULL);
@@ -1374,8 +1403,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_dead_tuples enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1436,6 +1479,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_dead_tuples enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1516,7 +1590,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;
@@ -1593,6 +1667,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->subretaindeadtuples &&
+ !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 f0b37a32ac1..d8d929c252b 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4100,7 +4105,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4315,6 +4321,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4458,6 +4468,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4539,6 +4556,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4584,12 +4622,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4597,9 +4644,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4632,6 +4736,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8605776ad86..6a7d2076385 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"},
};
/*
@@ -1665,6 +1666,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();
}
@@ -1783,6 +1789,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s,
}
}
+ if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION)
+ {
+ if (IsSlotForConflictCheck(NameStr(s->data.name)))
+ return RS_INVAL_CONFLICT_RETENTION_DURATION;
+ }
+
return RS_INVAL_NONE;
}
@@ -2044,6 +2056,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 d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 118d6da1ace..c284b4d2c6d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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 e8fc342d1a9..2f459404c9e 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 7c0204dd6f4..5bcd5c84844 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining conflict
+ * information. This is used only when retain_dead_tuples 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 6509fda77a9..30dafcfe10c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2169,9 +2169,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..aa6ef19dd70 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Fri, Aug 1, 2025 at 9:16 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Friday, August 1, 2025 7:42 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Fri, Aug 1, 2025 at 5:02 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:2.
+ If set to <literal>true</literal>, the detection of + <xref linkend="conflict-update-deleted"/> is enabled, and + a physical + replication slot named <quote><literal>pg_conflict_detection</literal></quote> created on the subscriber to prevent the conflict informationfrom
being removed.
"to prevent the conflict information from being removed." should be
rewritten as "to prevent removal of tuple required for conflict detection"It appears the document you commented is already committed. I think
the intention was to make a general statement that neither dead tuples
nor commit timestamp data would be removed.Okay got it, so instead of "conflict information" should we say "information for
detecting conflicts" or "conflict detection information", conflict information
looks like we want to prevent the information about the conflict which has
already happened, instead we are preventing information which are required
for detecting the conflict, does this make sense?It makes sense to me, so changed.
I know this is already committed, but actually this is part of the whole patch set
so we can always improvise it.3. + /* Return if the commit timestamp data is not available */ + if (!track_commit_timestamp) + return false;Shouldn't caller should take care of this? I mean if the
'retaindeadtuples' and 'track_commit_timestamp' is not set then
caller shouldn't even call this function.I feel moving the checks into a single central function would
streamline the caller, reducing code duplication. So, maybe we could
move the retaindeadtuple check into this function as well for consistency.Thoughts ?
Fine with either way, actually I wanted both the check 'retaindeadtuple' and
'track_commit_timestamp' at the same place.Thanks for confirming. Here is V56 patch set which addressed all the
comments including the comments from Amit[1] and Shveta[2].I have merged V55-0002 into 0001 and updated the list of author
and reviewers based on my knowledge.
Now this LGTM, I would suggest some modification in the commit
message, I think the purpose of this improvement is not very clear to
me, it says "supports detecting update_deleted conflicts during update
operations". But I think we can be more clearer on when we want to
detect such conflicts, here is my proposal, feel free to edit/revise/
"This commit improves conflict detection for updates. When an update
arrives for a row that has been locally deleted, the subscriber now
performs a secondary scan to check for the recently deleted tuple.
This check uses the old column values from remote tuple to find
a match among tuples that are visible to 'SnapshotAny' but not yet
vacuumed away.
If a matching, deleted tuple is found, an update_deleted conflict is
raised. This provides a more robust and accurate conflict resolution
process, preventing unexpected behavior by correctly identifying cases
where a remote update clashes with a local deletion."
--
Regards,
Dilip Kumar
Google
On Fri, Aug 1, 2025 at 9:16 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Thanks for confirming. Here is V56 patch set which addressed all the
comments including the comments from Amit[1] and Shveta[2].I have merged V55-0002 into 0001 and updated the list of author
and reviewers based on my knowledge.
Thank You Hou-San for the patches. Please find a few initial comments on 002:
1)
src/sgml/system-views.sgml:
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> 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
+ <literal>pg_conflict_detection</literal>, which is created when
+ <link
linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
We can mention 'max_conflict_retention_duration' here i.e:
...has exceeded the maximum allowable limit of max_conflict_retention_duration.
2)
Shall we rename 'conflict_retention_exceeds_max_duration' as
'conflict_info_retention_exceeds_limit'? It is better to incorporate
'info' keyword, but then
'conflict_info_retention_exceeds_max_duration' becomes too long and
thus I suggest 'conflict_info_retention_exceeds_limit'. Thoughts?
3)
src/sgml/monitoring.sgml:
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link
linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is
+ retained for conflict detection by this apply worker does not exceed
+ <link linkend="guc-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>;
NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
a)
In the html file, the link does not take me to
'max_conflict_retention_duration' GUC. It takes to that page but to
some other location.
b)
+ the duration for which conflict information is
+ retained for conflict detection by this apply worker
Shall this be better: 'the duration for which information useful for
conflict detection is retained by this apply worker'
4)
src/sgml/config.sgml:
a)
+ Maximum duration for which each apply worker can request to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions.
Shall it be :
"Maximum duration for which each apply worker is allowed to retain.."
or "can retain"?
b)
src/sgml/config.sgml
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that conflict information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
'that conflict information is retained' --> 'that information useful
for conflict detection is retained'
c)
src/sgml/config.sgml
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
'that used' --> 'that is used'
5)
ApplyLauncherMain():
+ /*
+ * Stop the conflict information retention only if all workers
+ * for subscriptions with retain_dead_tuples enabled have
+ * requested it.
+ */
+ stop_retention &= sub->enabled;
This comment is not clear. By enabling or disabling subscription, how
can it request for 'stop or continue conflict info retention'?
Do you mean we can not 'invalidate the slot' and thus stop retention
if a sub with rdt=ON is disabled? If so, we can pair it up with the
previous comment itself where we have mentioned that we can not
advance xmin when sub is disabled, as that comment indicates a clear
reason too.
6)
Above brings me to a point that in doc, shall we mention that if a sub
with rdt=on is disabled, even 'max_conflict_retention_duration' is not
considered for other subs which have rdt=ON.
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?
8)
+ * nonremovable xid. Similarly, stop the conflict information
+ * retention only if all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
Shall we rephrase to:
Similarly, can't stop the conflict information retention unless all
such workers are running.
thanks
Shveta
On Mon, Aug 4, 2025 at 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?
Before bikeshedding on the name of this option, I would like us to
once again consider whether we should provide this option at
subscription-level or GUC?
The rationale behind considering it as a subscription option is that
the different subscriptions may have different requirements for dead
tuple retention which means that for some particular subscription, the
workload may not be always high which means that even if temporarily
the lag_duration (of apply) has exceeded the new option's value, it
should become okay. So, in such a case users may not want to configure
max_conflict_retention_duration for a subscription which would
otherwise lead to stop detection of update_deleted conflict for that
subscription.
The other point is that it is only related to the retain_dead_tuples
option of the subscription, so providing this new option at the same
level would appear consistent.
I remember that previously Sawada-San has advocated it to provide as
GUC but I think the recent tests suggest that users should define
pub-sub topology carefuly to enable retain_dead_tuples option as even
mentioned in docs[2]https://www.postgresql.org/docs/devel/sql-createsubscription.html, so, it is worth considering to provide it at
subscription-level.
Thoughts?
[1]: /messages/by-id/CAD21AoCbjVTjejQxBkyo9kop2HMw85wSJqpB=JapsSE+Kw_iRg@mail.gmail.com
[2]: https://www.postgresql.org/docs/devel/sql-createsubscription.html
--
With Regards,
Amit Kapila.
On Monday, August 4, 2025 2:16 PM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, Aug 1, 2025 at 9:16 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Thanks for confirming. Here is V56 patch set which addressed all the
comments including the comments from Amit[1] and Shveta[2].I have merged V55-0002 into 0001 and updated the list of author and
reviewers based on my knowledge.Thank You Hou-San for the patches. Please find a few initial comments on 002:
1) src/sgml/system-views.sgml: + <para> + <literal>conflict_retention_exceeds_max_duration</literal> 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 + <literal>pg_conflict_detection</literal>, which is created when + <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literalretain_dead_tuples</literal></link>
+ is enabled.
+ </para>We can mention 'max_conflict_retention_duration' here i.e:
...has exceeded the maximum allowable limit of
max_conflict_retention_duration.
Added.
2)
Shall we rename 'conflict_retention_exceeds_max_duration' as
'conflict_info_retention_exceeds_limit'? It is better to incorporate 'info' keyword,
but then 'conflict_info_retention_exceeds_max_duration' becomes too long
and thus I suggest 'conflict_info_retention_exceeds_limit'. Thoughts?
I will think on this.
3) src/sgml/monitoring.sgml: + <row> + <entry role="catalog_table_entry"><para role="column_definition"> + <structfield>retain_dead_tuples</structfield> <type>boolean</type> + </para> + <para> + True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literalretain_dead_tuples</literal></link>
+ is enabled and the duration for which conflict information is + retained for conflict detection by this apply worker does not exceed + <link + linkend="guc-max-conflict-retention-duration"><literal>max_conflict_re + tention_duration</literal></link>; NULL for + parallel apply workers and table synchronization workers. + </para></entry> + </row>a)
In the html file, the link does not take me to 'max_conflict_retention_duration'
GUC. It takes to that page but to some other location.b) + the duration for which conflict information is + retained for conflict detection by this apply workerShall this be better: 'the duration for which information useful for conflict
detection is retained by this apply worker'
Changed.
4)
src/sgml/config.sgml:a) + Maximum duration for which each apply worker can request to retain the + information useful for conflict detection when + <literal>retain_dead_tuples</literal> is enabled for the associated + subscriptions.Shall it be :
"Maximum duration for which each apply worker is allowed to retain.."
or "can retain"?
Changed to "is allowed to"
b) src/sgml/config.sgml + subscriptions. The default value is <literal>0</literal>, indicating + that conflict information is retained until it is no longer needed for + detection purposes. If this value is specified without units, it is + taken as milliseconds.'that conflict information is retained' --> 'that information useful for conflict
detection is retained'
I changed to "the information" because the nearby texts have already
mentioned the usage of this information.
c) src/sgml/config.sgml + The replication slot + <quote><literal>pg_conflict_detection</literal></quote> that used to + retain conflict information will be invalidated if all apply workers + associated with the subscriptions, where'that used' --> 'that is used'
Fixed.
5) ApplyLauncherMain(): + /* + * Stop the conflict information retention only if all workers + * for subscriptions with retain_dead_tuples enabled have + * requested it. + */ + stop_retention &= sub->enabled;This comment is not clear. By enabling or disabling subscription, how can it
request for 'stop or continue conflict info retention'?Do you mean we can not 'invalidate the slot' and thus stop retention if a sub
with rdt=ON is disabled?
Yes, there is no apply worker for disabled
subscription, thus no way to request that.
If so, we can pair it up with the previous comment
itself where we have mentioned that we can not advance xmin when sub is
disabled, as that comment indicates a clear reason too.
Changed.
6)
Above brings me to a point that in doc, shall we mention that if a sub with
rdt=on is disabled, even 'max_conflict_retention_duration' is not considered
for other subs which have rdt=ON.
I think the documentation specifies that only active apply workers can make such
requests, which appears sufficient to me.
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more clear?
I will think on it.
8) + * nonremovable xid. Similarly, stop the conflict information + * retention only if all workers for subscriptions with + * retain_dead_tuples enabled have requested it.Shall we rephrase to:
Similarly, can't stop the conflict information retention unless all such workers
are running.
Changed.
Here is V57 patch set which addressed most of comments.
In this version, I also fixed a bug that the apply worker continued to
find dead tuples even if it has already stop retaining dead tuples.
Best Regards,
Hou zj
Attachments:
v57-0002-Re-create-the-replication-slot-if-the-conflict-r.patchapplication/octet-stream; name=v57-0002-Re-create-the-replication-slot-if-the-conflict-r.patchDownload
From 96454c0a39f1c7f72974ff3e4da25115e15f38d8 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 14:41:37 +0800
Subject: [PATCH v57 2/2] 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 | 15 ++++-
src/backend/replication/logical/worker.c | 64 +++++++++++++++-------
src/backend/utils/adt/pg_upgrade_support.c | 2 +-
src/include/replication/logicallauncher.h | 2 +-
5 files changed, 63 insertions(+), 25 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 94e68eaf588..b29d7727c3b 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5425,7 +5425,10 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>max_conflict_retention_duration</literal>. If the replication
slot is invalidated, you can disable
<literal>retain_dead_tuples</literal> and re-enable it after
- confirming this replication slot has been dropped.
+ confirming this replication slot has been dropped. Alternatively, the
+ invalidated slot will be automatically dropped and re-created once the
+ apply worker confirms that the retention duration is within the
+ specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 7b6ddda4dd1..1722b4089d2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1249,7 +1249,7 @@ ApplyLauncherMain(Datum main_arg)
* applying remote changes that occurred before the
* subscription was enabled.
*/
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(!sub->enabled);
}
if (!sub->enabled)
@@ -1286,6 +1286,12 @@ ApplyLauncherMain(Datum main_arg)
{
can_advance_xmin = false;
stop_retention = false;
+
+ /*
+ * Re-create the slot if it has been invalidated, as retention
+ * has now resumed.
+ */
+ CreateConflictDetectionSlot(true);
}
/*
@@ -1512,10 +1518,15 @@ invalidate_conflict_slot(void)
* conflict detection, if not yet.
*/
void
-CreateConflictDetectionSlot(void)
+CreateConflictDetectionSlot(bool recreate_if_invalid)
{
TransactionId xmin_horizon;
+ /* Drop the invalidated slot and re-create it if requested */
+ if (recreate_if_invalid && MyReplicationSlot &&
+ MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION)
+ ReplicationSlotDropAcquired();
+
/* Exit early, if the replication slot is already created and acquired */
if (MyReplicationSlot)
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index e01b0808d83..21825b1973d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -592,6 +592,8 @@ static void apply_handle_tuple_routing(ApplyExecutionData *edata,
LogicalRepTupleData *newtup,
CmdType operation);
+static void apply_worker_exit(void);
+
/* Functions for skipping changes */
static void maybe_start_skipping_changes(XLogRecPtr finish_lsn);
static void stop_skipping_changes(void);
@@ -4339,10 +4341,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (MyLogicalRepWorker->stop_conflict_info_retention)
- return false;
-
return true;
}
@@ -4624,6 +4622,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message and exit. This
+ * allows the launcher to recreate the replication slot prior to
+ * restarting the worker.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will restart to resume retaining conflict information",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ apply_worker_exit();
+ }
+
/*
* Reaching here means the remote WAL position has been received, and all
* transactions up to that position on the publisher have been applied and
@@ -4631,6 +4648,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4673,9 +4691,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_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_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4705,19 +4722,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
diff --git a/src/backend/utils/adt/pg_upgrade_support.c b/src/backend/utils/adt/pg_upgrade_support.c
index a4f8b4faa90..e20fc44adda 100644
--- a/src/backend/utils/adt/pg_upgrade_support.c
+++ b/src/backend/utils/adt/pg_upgrade_support.c
@@ -423,7 +423,7 @@ binary_upgrade_create_conflict_detection_slot(PG_FUNCTION_ARGS)
{
CHECK_IS_BINARY_UPGRADE;
- CreateConflictDetectionSlot();
+ CreateConflictDetectionSlot(false);
ReplicationSlotRelease();
diff --git a/src/include/replication/logicallauncher.h b/src/include/replication/logicallauncher.h
index 6e3007db5f0..5052c394c8f 100644
--- a/src/include/replication/logicallauncher.h
+++ b/src/include/replication/logicallauncher.h
@@ -29,7 +29,7 @@ extern void ApplyLauncherWakeupAtCommit(void);
extern void ApplyLauncherWakeup(void);
extern void AtEOXact_ApplyLauncher(bool isCommit);
-extern void CreateConflictDetectionSlot(void);
+extern void CreateConflictDetectionSlot(bool recreate_if_invalid);
extern bool IsLogicalLauncher(void);
--
2.50.1.windows.1
v57-0001-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v57-0001-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 4daed5f8e6570d828e20c02860ee26f0af880c0f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v57 1/2] 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_dead_tuples 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_dead_tuples 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_dead_tuples 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_dead_tuples' 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_dead_tuples 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 | 43 ++++++
doc/src/sgml/monitoring.sgml | 13 ++
doc/src/sgml/system-views.sgml | 12 ++
src/backend/catalog/system_views.sql | 3 +-
src/backend/replication/logical/launcher.c | 129 ++++++++++++++---
src/backend/replication/logical/worker.c | 134 ++++++++++++++++--
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 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
15 files changed, 359 insertions(+), 34 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..94e68eaf588 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,49 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker is allowed to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> that is used to
+ retain conflict information will be invalidated if all apply workers
+ associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. If the replication
+ slot is invalidated, you can disable
+ <literal>retain_dead_tuples</literal> and re-enable it after
+ confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index fa78031ccbb..0c22eac26da 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which information useful for conflict
+ detection is retained by this apply worker does not exceed
+ <xref linkend="guc-max-conflict-retention-duration"/>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/system-views.sgml b/doc/src/sgml/system-views.sgml
index 4187191ea74..a3a290900b2 100644
--- a/doc/src/sgml/system-views.sgml
+++ b/doc/src/sgml/system-views.sgml
@@ -3007,6 +3007,18 @@ SELECT * FROM pg_locks pl LEFT JOIN pg_prepared_xacts ppx
<xref linkend="guc-idle-replication-slot-timeout"/> duration.
</para>
</listitem>
+ <listitem>
+ <para>
+ <literal>conflict_retention_exceeds_max_duration</literal> means that
+ the duration for retaining conflict information, which is used
+ in logical replication conflict detection, has exceeded the maximum
+ allowable limit configured by <xref linkend="guc-max-conflict-retention-duration"/>.
+ It is set only for the slot <literal>pg_conflict_detection</literal>,
+ which is created when
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled.
+ </para>
+ </listitem>
</itemizedlist>
</para></entry>
</row>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 77c693f630e..235b6b4f323 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 37377f7eb63..7b6ddda4dd1 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;
@@ -100,9 +103,11 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention);
static bool acquire_conflict_slot_if_exists(void);
static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void invalidate_conflict_slot(void);
/*
@@ -465,6 +470,8 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ MyReplicationSlot->data.invalidated != RS_INVAL_NONE);
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1183,6 +1190,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool stop_retention = true;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1215,15 +1223,17 @@ ApplyLauncherMain(Datum main_arg)
retain_dead_tuples = true;
/*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
+ * Can't advance xmin of the slot or stop conflict information
+ * retention unless all the subscriptions with
+ * retain_dead_tuples are enabled. This is required to ensure
+ * that we don't advance the xmin of CONFLICT_DETECTION_SLOT
+ * if one of the subscriptions is not enabled. Otherwise, we
+ * won't be able to detect conflicts reliably for such a
+ * subscription even though it has set the retain_dead_tuples
+ * option.
*/
can_advance_xmin &= sub->enabled;
+ stop_retention &= sub->enabled;
/*
* Create a replication slot to retain information necessary
@@ -1256,8 +1266,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin,
+ &stop_retention);
/* worker is running already */
continue;
@@ -1267,10 +1278,15 @@ ApplyLauncherMain(Datum main_arg)
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions with retain_dead_tuples are
* running, disabling the further computation of the minimum
- * nonremovable xid.
+ * nonremovable xid. Similarly, can't stop the conflict
+ * information retention unless all workers for subscriptions with
+ * retain_dead_tuples enabled have requested it.
*/
if (sub->retaindeadtuples)
+ {
can_advance_xmin = false;
+ stop_retention = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1316,13 +1332,16 @@ ApplyLauncherMain(Datum main_arg)
}
/*
- * Drop the CONFLICT_DETECTION_SLOT slot if there is no subscription
- * that requires us to retain dead tuples. Otherwise, if required,
- * advance the slot's xmin to protect dead tuples required for the
- * conflict detection.
+ * Invalidate the slot if requested by all apply workers. Or, drop the
+ * CONFLICT_DETECTION_SLOT slot if there is no subscription that
+ * requires us to retain dead tuples. Otherwise, if required, advance
+ * the slot's xmin to protect dead tuples required for the conflict
+ * detection.
*/
if (MyReplicationSlot)
{
+ if (retain_dead_tuples && stop_retention)
+ invalidate_conflict_slot();
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
@@ -1360,11 +1379,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * if any worker continues retaining conflict information, *stop_retention is
+ * set to false.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin, bool *stop_retention)
{
TransactionId nonremovable_xid;
+ bool stop_conflict_info_retention;
Assert(worker != NULL);
@@ -1376,8 +1400,22 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_conflict_info_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
+ /*
+ * Stop the conflict information retention only if all workers for
+ * subscriptions with retain_dead_tuples enabled have requested it.
+ */
+ *stop_retention &= stop_conflict_info_retention;
+
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention or if advancement is not possible.
+ */
+ if (stop_conflict_info_retention || !can_advance_xmin)
+ return;
+
Assert(TransactionIdIsValid(nonremovable_xid));
if (!TransactionIdIsValid(*xmin) ||
@@ -1438,6 +1476,37 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
return;
}
+/*
+ * Invalidate the replication slot used to retain information for conflict
+ * detection.
+ */
+static void
+invalidate_conflict_slot(void)
+{
+ Assert(MyReplicationSlot);
+
+ /*
+ * Do nothing if the replication slot has already been invalidated due to
+ * conflict retention duration.
+ */
+ if (MyReplicationSlot->data.invalidated != RS_INVAL_NONE)
+ {
+ Assert(MyReplicationSlot->data.invalidated == RS_INVAL_CONFLICT_RETENTION_DURATION);
+ return;
+ }
+
+ ReplicationSlotRelease();
+ InvalidateObsoleteReplicationSlots(RS_INVAL_CONFLICT_RETENTION_DURATION,
+ InvalidXLogRecPtr, InvalidOid,
+ InvalidTransactionId);
+
+ /*
+ * Acquire the invalidated slot to allow the launcher to drop it in the
+ * next cycle if no subscriptions have retain_dead_tuples enabled.
+ */
+ ReplicationSlotAcquire(CONFLICT_DETECTION_SLOT, true, false);
+}
+
/*
* Create and acquire the replication slot used to retain information for
* conflict detection, if not yet.
@@ -1518,7 +1587,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;
@@ -1595,6 +1664,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->subretaindeadtuples &&
+ !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 89e241c8392..e01b0808d83 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3228,6 +3233,14 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Return false if the current apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -4110,7 +4123,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (!MyLogicalRepWorker->stop_conflict_info_retention &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4339,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ return false;
+
return true;
}
@@ -4468,6 +4486,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4574,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4640,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4662,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to invalidate the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4642,6 +4754,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/replication/slot.c b/src/backend/replication/slot.c
index 8605776ad86..6a7d2076385 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"},
};
/*
@@ -1665,6 +1666,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();
}
@@ -1783,6 +1789,12 @@ DetermineSlotInvalidationCause(uint32 possible_causes, ReplicationSlot *s,
}
}
+ if (possible_causes & RS_INVAL_CONFLICT_RETENTION_DURATION)
+ {
+ if (IsSlotForConflictCheck(NameStr(s->data.name)))
+ return RS_INVAL_CONFLICT_RETENTION_DURATION;
+ }
+
return RS_INVAL_NONE;
}
@@ -2044,6 +2056,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 d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 118d6da1ace..c284b4d2c6d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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 e8fc342d1a9..2f459404c9e 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 7c0204dd6f4..5bcd5c84844 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -97,6 +97,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining conflict
+ * information. This is used only when retain_dead_tuples 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 6509fda77a9..30dafcfe10c 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2169,9 +2169,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..aa6ef19dd70 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Tuesday, August 5, 2025 10:09 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
Here is V57 patch set which addressed most of comments.
In this version, I also fixed a bug that the apply worker continued to find dead
tuples even if it has already stop retaining dead tuples.
Here is a V58 patch set which improved few things by internal review:
0001:
* Remove the slot invalidation.
Initially, we thought it would be convenient for users to determine if they can
reliably detect update_deleted by checking the validity of the conflict
detection slot. However, after re-thinking, even if the slot is valid, it
doesn't guarantee that each apply worker can reliably detect conflicts. Some
apply workers might have stopped retention, yet the slot remains valid due to
other active workers continuing retention.
Instead of querying the slot, users should verify the ability of a specific
apply worker to reliably detect conflicts by checking the view
pg_stat_subscription.retain_dead_tuples.
So, slot invalidation would be necessary. We could set slot.xmin to invalid
instead to allow dead tuples to be removed when all apply workers stop
retention. This approach simplifies implementation and avoids introducing a new
invalidation type solely for one internal slot.
* Fixed a bug that parallel apply worker continues to search dead tuples when
the retention has already stopped. The parallel and table sync worker referred
to its own stop_conflict_info_retention flag, but should refer to the
retention flag of the leader instead because only leader mananges this flag.
0002:
* Allow the apply worker to wait for the slot to be recover after resuming the dead
tuple retention instead of restarting the apply worker.
Best Regards,
Hou zj
Attachments:
v58-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v58-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 86d7a437f737e3976ad28dcf8056b08ec64c87b4 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 6 Aug 2025 19:02:56 +0800
Subject: [PATCH v58 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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 | 7 +-
src/backend/replication/logical/launcher.c | 83 +++++++++++++----
src/backend/replication/logical/worker.c | 100 ++++++++++++++++-----
src/include/replication/worker_internal.h | 7 ++
4 files changed, 158 insertions(+), 39 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 5a9aee66e3f..6cd0b48c8dd 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5421,8 +5421,11 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration exceeded the
<literal>max_conflict_retention_duration</literal>. To re-enable
- retention, you can disable <literal>retain_dead_tuples</literal> and
- re-enable it after confirming this replication slot has been dropped.
+ retention manually, you can disable <literal>retain_dead_tuples</literal>
+ and re-enable it after confirming this replication slot has been dropped.
+ Alternatively, the retention will be automatically resumed
+ once at least one apply worker confirms that the retention duration is
+ within the specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 03aa5b2b7a9..c3ffff0fdb2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -103,9 +103,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -465,6 +468,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
@@ -1259,8 +1264,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1370,9 +1375,11 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool stop_retention;
Assert(worker != NULL);
@@ -1384,13 +1391,43 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->relmutex);
/*
* Skip collecting oldest_nonremovable_xid for workers that have stopped
* conflict retention.
*/
+ if (stop_retention)
+ return;
+
+ /*
+ * Initialize slot.xmin as a apply worker resumes retention of information
+ * critical for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see get_candidate_xid).
+ */
if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
@@ -1452,23 +1489,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1488,6 +1517,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index e327ffe8e00..72deb83c228 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -418,6 +418,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -4375,10 +4379,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4416,6 +4416,33 @@ get_candidate_xid(RetainDeadTuplesData *rdt_data)
TransactionId oldest_running_xid;
TimestampTz now;
+ /*
+ * No need to advance if the apply worker has resumed retention but the
+ * launcher has not yet initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ rdt_data->wait_for_initial_xid = false;
+ }
+
/*
* Use last_recv_time when applying changes in the loop to avoid
* unnecessary system time retrieval. If last_recv_time is not available,
@@ -4660,13 +4687,38 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message. The next step is
+ * to wait for the launcher to initialize the oldest_nonremovable_xid.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" resumes retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ rdt_data->wait_for_initial_xid = true;
+ }
+
/*
* 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.
+ *
+ * However, if oldest_nonremovable_xid is invalid, indicating that
+ * retention was stopped and is now being resumed, refrain from updating
+ * oldest_nonremovable_xid until the launcher provides an initial value
+ * (see get_candidate_xid() for details).
*/
SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->relmutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4709,9 +4761,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* to InvalidTransactionId, notify the launcher to set the slot.xmin to
* InvalidTransactionId as well, and return true. Return false otherwise.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4741,18 +4792,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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 = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- /* Notify launcher to update 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 the information for detecting conflicts",
+ 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 = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
@@ -5607,6 +5666,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ !MyLogicalRepWorker->stop_conflict_info_retention &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b86c759394f..54a55e7c1bd 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -100,6 +100,13 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining information
+ * useful for conflict detection. This is used only when
+ * retain_dead_tuples is enabled.
+ */
+ bool stop_conflict_info_retention;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.50.1.windows.1
v58-0001-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v58-0001-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From ced59673bf4b88d3ed80ee250f398412f04f4214 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v58 1/2] 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_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it after confirming that the
replication slot has been dropped. An upcoming patch will include support for
automatic slot re-initialization 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_dead_tuples' 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_dead_tuples 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 | 40 ++++
doc/src/sgml/monitoring.sgml | 13 ++
src/backend/catalog/system_views.sql | 3 +-
src/backend/replication/logical/launcher.c | 40 +++-
src/backend/replication/logical/worker.c | 171 +++++++++++++++++-
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/worker_internal.h | 3 +
src/test/regress/expected/rules.out | 5 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
12 files changed, 280 insertions(+), 26 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..5a9aee66e3f 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,46 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker is allowed to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. To re-enable
+ retention, you can disable <literal>retain_dead_tuples</literal> and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..cc8baf97223 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>retain_dead_tuples</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which information useful for conflict
+ detection is retained by this apply worker does not exceed
+ <xref linkend="guc-max-conflict-retention-duration"/>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..36928430ff3 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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_dead_tuples
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 37377f7eb63..03aa5b2b7a9 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;
@@ -102,7 +105,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -1320,13 +1323,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention, the
+ * new xmin will be set to InvalidTransactionId. We then update
+ * slot.xmin accordingly to permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1378,7 +1386,12 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1415,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1531,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;
@@ -1595,6 +1608,15 @@ 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))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ 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 0fdc5de57ba..e327ffe8e00 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3225,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool stop_retention;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3234,42 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ */
+ if (am_leader_apply_worker())
+ {
+ stop_retention =
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->relmutex);
+ stop_retention = !TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (stop_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3296,14 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This occurs when the leader and all other apply
+ * workers cease retention immediately after obtaining the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4159,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4375,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4468,6 +4522,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4610,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4676,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4698,65 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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->oldest_nonremovable_xid
+ * to InvalidTransactionId, notify the launcher to set the slot.xmin to
+ * InvalidTransactionId as well, and return true. Return false otherwise.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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 = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4642,6 +4789,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c
index d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 118d6da1ace..c284b4d2c6d 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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_dead_tuples}',
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 b29453e8e4f..6e3007db5f0 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/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..b86c759394f 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..35c24285dd9 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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_dead_tuples
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_dead_tuples) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..aa6ef19dd70 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT retain_dead_tuples FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Mon, Aug 4, 2025 at 3:11 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 4, 2025 at 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?Before bikeshedding on the name of this option, I would like us to
once again consider whether we should provide this option at
subscription-level or GUC?The rationale behind considering it as a subscription option is that
the different subscriptions may have different requirements for dead
tuple retention which means that for some particular subscription, the
workload may not be always high which means that even if temporarily
the lag_duration (of apply) has exceeded the new option's value, it
should become okay. So, in such a case users may not want to configure
max_conflict_retention_duration for a subscription which would
otherwise lead to stop detection of update_deleted conflict for that
subscription.
Yes valid point, and it's also possible that for some subscription
user is okay to not retain dead tuple if it crosses certain duration
OTOH for some subscription it is too critical to retain dead tuple
even if user has to take some performance hit, so might want to have
higher threshold for those slots.
The other point is that it is only related to the retain_dead_tuples
option of the subscription, so providing this new option at the same
level would appear consistent.
Yes that's a valid argument, because if the user is setting retain
dead tuples for subscription then only they need to consider setting
duration.
I remember that previously Sawada-San has advocated it to provide as
GUC but I think the recent tests suggest that users should define
pub-sub topology carefuly to enable retain_dead_tuples option as even
mentioned in docs[2], so, it is worth considering to provide it at
subscription-level.
IMHO, it should be fine to provide the subscription option first and
if we see complaints about inconvenience we may consider GUC as well
in the future.
--
Regards,
Dilip Kumar
Google
On Thu, Aug 7, 2025 at 6:05 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Aug 4, 2025 at 3:11 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 4, 2025 at 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?Before bikeshedding on the name of this option, I would like us to
once again consider whether we should provide this option at
subscription-level or GUC?The rationale behind considering it as a subscription option is that
the different subscriptions may have different requirements for dead
tuple retention which means that for some particular subscription, the
workload may not be always high which means that even if temporarily
the lag_duration (of apply) has exceeded the new option's value, it
should become okay. So, in such a case users may not want to configure
max_conflict_retention_duration for a subscription which would
otherwise lead to stop detection of update_deleted conflict for that
subscription.Yes valid point, and it's also possible that for some subscription
user is okay to not retain dead tuple if it crosses certain duration
OTOH for some subscription it is too critical to retain dead tuple
even if user has to take some performance hit, so might want to have
higher threshold for those slots.The other point is that it is only related to the retain_dead_tuples
option of the subscription, so providing this new option at the same
level would appear consistent.Yes that's a valid argument, because if the user is setting retain
dead tuples for subscription then only they need to consider setting
duration.I remember that previously Sawada-San has advocated it to provide as
GUC but I think the recent tests suggest that users should define
pub-sub topology carefuly to enable retain_dead_tuples option as even
mentioned in docs[2], so, it is worth considering to provide it at
subscription-level.IMHO, it should be fine to provide the subscription option first and
if we see complaints about inconvenience we may consider GUC as well
in the future.
+1
thanks
Shveta
On Thu, Aug 7, 2025 at 10:10 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, August 5, 2025 10:09 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
Here is V57 patch set which addressed most of comments.
In this version, I also fixed a bug that the apply worker continued to find dead
tuples even if it has already stop retaining dead tuples.Here is a V58 patch set which improved few things by internal review:
0001:
Thank You for the patches, please find a few comments on 001 alone:
1)
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration).
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
We can add comments here as in why we are adding table-sync time to
max_conflict_retention_duration.
2)
relmutex comment says:
/* Used for initial table synchronization. */
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
slock_t relmutex;
We shall update this comment as now we are using it for other
purposes. Also name is specific to relation (due to originally created
for table-sync case). We can rename it to be more general so that it
can be used for oldest-xid access purposes as well.
3)
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!max_conflict_retention_duration)
+ return false;
Shall we move 'max_conflict_retention_duration' NULL check as the
first step. Or do you think it will be better to move it to the caller
before should_stop_conflict_info_retention is invoked?
4)
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. To re-enable
+ retention, you can disable <literal>retain_dead_tuples</literal> and
+ re-enable it after confirming this replication slot has been dropped.
But the replication slot will not be dropped unless all the
subscriptions have disabled retain_dead_tuples. So shall the above doc
somehow mention this part as well otherwise it could be misleading for
users.
5)
pg_stat_subscription_stats: retain_dead_tuples
Can it cause confusion as both subscription's parameter and
pg_stat_subscription_stats's column have the same name while may have
different values. Shall the stats one be named as
'effective_retain_dead_tuples'?
thanks
Shveta
On Friday, August 8, 2025 2:34 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Aug 7, 2025 at 10:10 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, August 5, 2025 10:09 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is V57 patch set which addressed most of comments.
In this version, I also fixed a bug that the apply worker continued
to find dead tuples even if it has already stop retaining dead tuples.Here is a V58 patch set which improved few things by internal review:
0001:
Thank You for the patches, please find a few comments on 001 alone:
Thanks for the comments.
1) + /* + * Return if the wait time has not exceeded the maximum limit + * (max_conflict_retention_duration). + */ + if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now, + max_conflict_retention_duration + + rdt_data->table_sync_wait_time))We can add comments here as in why we are adding table-sync time to
max_conflict_retention_duration.
Added.
2)
relmutex comment says:/* Used for initial table synchronization. */
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
slock_t relmutex;We shall update this comment as now we are using it for other purposes. Also
name is specific to relation (due to originally created for table-sync case). We
can rename it to be more general so that it can be used for oldest-xid access
purposes as well.
Changed the name and added comments.
3) + Assert(TransactionIdIsValid(rdt_data->candidate_xid)); + Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS || + rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH); + + if (!max_conflict_retention_duration) + return false;Shall we move 'max_conflict_retention_duration' NULL check as the first step.
Or do you think it will be better to move it to the caller before
should_stop_conflict_info_retention is invoked?
I think these Asserts are good to have, even if the GUC is not
specified, so I kept the current style.
4) + The information useful for conflict detection is no longer retained if + all apply workers associated with the subscriptions, where + <literal>retain_dead_tuples</literal> is enabled, confirm that the + retention duration exceeded the + <literal>max_conflict_retention_duration</literal>. To re-enable + retention, you can disable <literal>retain_dead_tuples</literal> and + re-enable it after confirming this replication slot has been dropped.But the replication slot will not be dropped unless all the subscriptions have
disabled retain_dead_tuples. So shall the above doc somehow mention this
part as well otherwise it could be misleading for users.
Added.
5)
pg_stat_subscription_stats: retain_dead_tuplesCan it cause confusion as both subscription's parameter and
pg_stat_subscription_stats's column have the same name while may have
different values. Shall the stats one be named as
'effective_retain_dead_tuples'?
I think the prefix "effective_" is typically used for non-boolean options (such
as effective_cache_size or effective_io_concurrency). So, I opted for the name
"dead_tuple_retention_active" as it aligns with some existing names like
"row_security_active."
Here is V59 patch set which addressed above comments in 0001.
Best Regards,
Hou zj
Attachments:
v59-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v59-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 29f8614986172b30e3fb4f35a07b5b7ad8252cde Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 8 Aug 2025 17:05:00 +0800
Subject: [PATCH v59 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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 | 8 +-
src/backend/replication/logical/launcher.c | 83 +++++++++++++----
src/backend/replication/logical/worker.c | 100 ++++++++++++++++-----
src/include/replication/worker_internal.h | 7 ++
4 files changed, 158 insertions(+), 40 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 95772fe44b2..0e5497591cd 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5427,9 +5427,11 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration exceeded the
<literal>max_conflict_retention_duration</literal>. To re-enable
- retention, you can disable <literal>retain_dead_tuples</literal> for all
- subscriptions and re-enable it after confirming this replication slot has
- been dropped.
+ retention manually, you can disable <literal>retain_dead_tuples</literal>
+ for all subscriptions and re-enable it after confirming this replication
+ slot has been dropped. Alternatively, the retention will be automatically
+ resumed once at least one apply worker confirms that the retention
+ duration is within the specified limit.
</para>
<para>
This option is effective only if a subscription with
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c5002effaee..ee37fcb4057 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -103,9 +103,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -465,6 +468,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
@@ -1259,8 +1264,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1370,9 +1375,11 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool stop_retention;
Assert(worker != NULL);
@@ -1384,13 +1391,43 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->mutex);
/*
* Skip collecting oldest_nonremovable_xid for workers that have stopped
* conflict retention.
*/
+ if (stop_retention)
+ return;
+
+ /*
+ * Initialize slot.xmin as a apply worker resumes retention of information
+ * critical for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see get_candidate_xid).
+ */
if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->mutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->mutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
@@ -1452,23 +1489,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1488,6 +1517,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5fb34cb20d6..1bc917679bc 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -418,6 +418,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -4376,10 +4380,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4417,6 +4417,33 @@ get_candidate_xid(RetainDeadTuplesData *rdt_data)
TransactionId oldest_running_xid;
TimestampTz now;
+ /*
+ * No need to advance if the apply worker has resumed retention but the
+ * launcher has not yet initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ rdt_data->wait_for_initial_xid = false;
+ }
+
/*
* Use last_recv_time when applying changes in the loop to avoid
* unnecessary system time retrieval. If last_recv_time is not available,
@@ -4661,13 +4688,38 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message. The next step is
+ * to wait for the launcher to initialize the oldest_nonremovable_xid.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" resumes retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ max_conflict_retention_duration));
+
+ rdt_data->wait_for_initial_xid = true;
+ }
+
/*
* 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.
+ *
+ * However, if oldest_nonremovable_xid is invalid, indicating that
+ * retention was stopped and is now being resumed, refrain from updating
+ * oldest_nonremovable_xid until the launcher provides an initial value
+ * (see get_candidate_xid() for details).
*/
SpinLockAcquire(&MyLogicalRepWorker->mutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4710,9 +4762,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* to InvalidTransactionId, notify the launcher to set the slot.xmin to
* InvalidTransactionId as well, and return true. Return false otherwise.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4744,18 +4795,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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->mutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->mutex);
-
- /* Notify launcher to update 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 the information for detecting conflicts",
+ 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->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
@@ -5610,6 +5669,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ !MyLogicalRepWorker->stop_conflict_info_retention &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9c0c2b8050c..32a1cbb6528 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -105,6 +105,13 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining information
+ * useful for conflict detection. This is used only when
+ * retain_dead_tuples is enabled.
+ */
+ bool stop_conflict_info_retention;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.50.1.windows.1
v59-0001-Introduce-a-new-GUC-max_conflict_retention_durat.patchapplication/octet-stream; name=v59-0001-Introduce-a-new-GUC-max_conflict_retention_durat.patchDownload
From 44889addf51f8e0a3491c17ae454305e2a5e1376 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v59] 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_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it after confirming that the
replication slot has been dropped. An upcoming patch will include support for
automatic slot re-initialization 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 'dead_tuple_retention_active' 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 dead_tuple_retention_active 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 ++
src/backend/catalog/system_views.sql | 3 +-
src/backend/replication/logical/launcher.c | 46 +++--
src/backend/replication/logical/tablesync.c | 22 +--
src/backend/replication/logical/worker.c | 177 ++++++++++++++++--
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/worker_internal.h | 10 +-
src/test/regress/expected/rules.out | 5 +-
src/test/subscription/t/035_conflicts.pl | 10 +-
13 files changed, 305 insertions(+), 43 deletions(-)
diff --git a/doc/src/sgml/config.sgml b/doc/src/sgml/config.sgml
index 20ccb2d6b54..9b59c0fd5b9 100644
--- a/doc/src/sgml/config.sgml
+++ b/doc/src/sgml/config.sgml
@@ -5399,6 +5399,47 @@ ANY <replaceable class="parameter">num_sync</replaceable> ( <replaceable class="
</listitem>
</varlistentry>
+ <varlistentry id="guc-max-conflict-retention-duration" xreflabel="max_conflict_retention_duration">
+ <term><varname>max_conflict_retention_duration</varname> (<type>integer</type>)
+ <indexterm>
+ <primary><varname>max_conflict_retention_duration</varname> configuration parameter</primary>
+ </indexterm>
+ </term>
+ <listitem>
+ <para>
+ Maximum duration for which each apply worker is allowed to retain the
+ information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. If this value is specified without units, it is
+ taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal>. To re-enable
+ retention, you can disable <literal>retain_dead_tuples</literal> for all
+ subscriptions and re-enable it after confirming this replication slot has
+ been dropped.
+ </para>
+ <para>
+ This option is effective only if a subscription with
+ <literal>retain_dead_tuples</literal> enabled is present, and the
+ associated apply worker is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ conflict information being removed prematurely, potentially missing
+ some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
+
</variablelist>
</sect2>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..e103bfd732d 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which information useful for conflict
+ detection is retained by this apply worker does not exceed
+ <xref linkend="guc-max-conflict-retention-duration"/>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..f654af0717e 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 37377f7eb63..c5002effaee 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;
@@ -102,7 +105,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -998,7 +1001,7 @@ ApplyLauncherShmemInit(void)
LogicalRepWorker *worker = &LogicalRepCtx->workers[slot];
memset(worker, 0, sizeof(LogicalRepWorker));
- SpinLockInit(&worker->relmutex);
+ SpinLockInit(&worker->mutex);
}
}
}
@@ -1320,13 +1323,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention, the
+ * new xmin will be set to InvalidTransactionId. We then update
+ * slot.xmin accordingly to permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1374,11 +1382,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
*/
Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
+ SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockRelease(&worker->mutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1415,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1531,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;
@@ -1595,6 +1608,15 @@ 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))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3356bc84ee..1ab5496f63f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -293,7 +293,7 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
@@ -305,7 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
MyLogicalRepWorker->relstate_lsn = current_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* UpdateSubscriptionRelState must be called within a transaction.
@@ -390,7 +390,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
finish_sync_worker();
}
else
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
}
/*
@@ -534,7 +534,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
if (syncworker)
{
/* Found one, update our copy of its state */
- SpinLockAcquire(&syncworker->relmutex);
+ SpinLockAcquire(&syncworker->mutex);
rstate->state = syncworker->relstate;
rstate->lsn = syncworker->relstate_lsn;
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -547,7 +547,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
syncworker->relstate_lsn =
Max(syncworker->relstate_lsn, current_lsn);
}
- SpinLockRelease(&syncworker->relmutex);
+ SpinLockRelease(&syncworker->mutex);
/* If we told worker to catch up, wait for it. */
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -1342,10 +1342,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
must_use_password = MySubscription->passwordrequired &&
!MySubscription->ownersuperuser;
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = relstate;
MyLogicalRepWorker->relstate_lsn = relstate_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* If synchronization is already done or no longer necessary, exit now
@@ -1428,10 +1428,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
goto copy_table_done;
}
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/* Update the state and make it visible to others. */
StartTransactionCommand();
@@ -1586,10 +1586,10 @@ copy_table_done:
/*
* We are done with the initial data synchronization, update the state.
*/
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCWAIT;
MyLogicalRepWorker->relstate_lsn = *origin_startpos;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* Finally, wait until the leader apply worker tells us to catch up and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0fdc5de57ba..6b5e218f603 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3225,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool stop_retention;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3234,42 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ */
+ if (am_leader_apply_worker())
+ {
+ stop_retention =
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->mutex);
+ stop_retention = !TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->mutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (stop_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3296,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4160,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4376,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4468,6 +4523,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4611,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4583,9 +4666,9 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rdt_data->remote_lsn),
@@ -4594,12 +4677,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4699,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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->oldest_nonremovable_xid
+ * to InvalidTransactionId, notify the launcher to set the slot.xmin to
+ * InvalidTransactionId as well, and return true. Return false otherwise.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ max_conflict_retention_duration +
+ rdt_data->table_sync_wait_time))
+ 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->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4642,6 +4791,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ max_conflict_retention_duration);
}
else
{
diff --git a/src/backend/utils/misc/guc_tables.c b/src/backend/utils/misc/guc_tables.c
index d14b1678e7f..580762c6a00 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 a9d8293474a..45c9e0a16a0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -396,6 +396,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 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
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 b29453e8e4f..6e3007db5f0 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/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..9c0c2b8050c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -64,7 +64,12 @@ typedef struct LogicalRepWorker
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
- slock_t relmutex;
+
+ /*
+ * Spinlock used to protect table synchronization information and the
+ * oldest_nonremovable_xid.
+ */
+ slock_t mutex;
/*
* Used to create the changes and subxact files for the streaming
@@ -94,6 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Thursday, August 7, 2025 8:35 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Aug 4, 2025 at 3:11 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Mon, Aug 4, 2025 at 11:46 AM shveta malik <shveta.malik@gmail.com>
wrote:
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?Before bikeshedding on the name of this option, I would like us to
once again consider whether we should provide this option at
subscription-level or GUC?The rationale behind considering it as a subscription option is that
the different subscriptions may have different requirements for dead
tuple retention which means that for some particular subscription, the
workload may not be always high which means that even if temporarily
the lag_duration (of apply) has exceeded the new option's value, it
should become okay. So, in such a case users may not want to configure
max_conflict_retention_duration for a subscription which would
otherwise lead to stop detection of update_deleted conflict for that
subscription.Yes valid point, and it's also possible that for some subscription user is okay to
not retain dead tuple if it crosses certain duration OTOH for some subscription
it is too critical to retain dead tuple even if user has to take some performance
hit, so might want to have higher threshold for those slots.The other point is that it is only related to the retain_dead_tuples
option of the subscription, so providing this new option at the same
level would appear consistent.Yes that's a valid argument, because if the user is setting retain dead tuples for
subscription then only they need to consider setting duration.I remember that previously Sawada-San has advocated it to provide as
GUC but I think the recent tests suggest that users should define
pub-sub topology carefuly to enable retain_dead_tuples option as even
mentioned in docs[2], so, it is worth considering to provide it at
subscription-level.IMHO, it should be fine to provide the subscription option first and if we see
complaints about inconvenience we may consider GUC as well in the future.
I agree. So, following the above points and some off-list discussions, I have
revised the option to be a subscription option in the V60 version.
Best Regards,
Hou zj
Attachments:
v60-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v60-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 59c2961b980e496a11bb24ab05b7f613e46d8f58 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 11 Aug 2025 16:23:24 +0800
Subject: [PATCH v60 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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/ref/create_subscription.sgml | 7 +-
src/backend/replication/logical/launcher.c | 83 +++++++++++++----
src/backend/replication/logical/worker.c | 100 ++++++++++++++++-----
src/include/replication/worker_internal.h | 7 ++
4 files changed, 158 insertions(+), 39 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 6121c86561e..0e34d136672 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -539,9 +539,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration exceeded the
<literal>max_conflict_retention_duration</literal> set within the
- corresponding subscription. To re-enable retention, you can disable
- <literal>retain_dead_tuples</literal> for all subscriptions and
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
re-enable it after confirming this replication slot has been dropped.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit.
</para>
<para>
Note that Overall retention will not stop if other subscriptions
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f1dfb51ccf8..b85acd95e6b 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -464,6 +467,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
@@ -1258,8 +1263,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1369,9 +1374,11 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool stop_retention;
Assert(worker != NULL);
@@ -1383,13 +1390,43 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->mutex);
/*
* Skip collecting oldest_nonremovable_xid for workers that have stopped
* conflict retention.
*/
+ if (stop_retention)
+ return;
+
+ /*
+ * Initialize slot.xmin as a apply worker resumes retention of information
+ * critical for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see get_candidate_xid).
+ */
if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->mutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->mutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
@@ -1451,23 +1488,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1487,6 +1516,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 5269cc980f2..246938f2011 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -418,6 +418,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -4376,10 +4380,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4417,6 +4417,33 @@ get_candidate_xid(RetainDeadTuplesData *rdt_data)
TransactionId oldest_running_xid;
TimestampTz now;
+ /*
+ * No need to advance if the apply worker has resumed retention but the
+ * launcher has not yet initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ rdt_data->wait_for_initial_xid = false;
+ }
+
/*
* Use last_recv_time when applying changes in the loop to avoid
* unnecessary system time retrieval. If last_recv_time is not available,
@@ -4661,13 +4688,38 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, log a message. The next step is
+ * to wait for the launcher to initialize the oldest_nonremovable_xid.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" resumes retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ MySubscription->maxconflretention));
+
+ rdt_data->wait_for_initial_xid = true;
+ }
+
/*
* 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.
+ *
+ * However, if oldest_nonremovable_xid is invalid, indicating that
+ * retention was stopped and is now being resumed, refrain from updating
+ * oldest_nonremovable_xid until the launcher provides an initial value
+ * (see get_candidate_xid() for details).
*/
SpinLockAcquire(&MyLogicalRepWorker->mutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
@@ -4710,9 +4762,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* to InvalidTransactionId, notify the launcher to set the slot.xmin to
* InvalidTransactionId as well, and return true. Return false otherwise.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4743,18 +4794,26 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
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.",
- MySubscription->maxconflretention));
-
- SpinLockAcquire(&MyLogicalRepWorker->mutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->mutex);
-
- /* Notify launcher to update 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 the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The time spent advancing the non-removable transaction ID has exceeded the maximum limit of %u ms.",
+ MySubscription->maxconflretention));
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+ }
reset_retention_data_fields(rdt_data);
@@ -5609,6 +5668,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ !MyLogicalRepWorker->stop_conflict_info_retention &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9c0c2b8050c..32a1cbb6528 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -105,6 +105,13 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining information
+ * useful for conflict detection. This is used only when
+ * retain_dead_tuples is enabled.
+ */
+ bool stop_conflict_info_retention;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.50.1.windows.1
v60-0001-Introduce-a-max_conflict_retention_duration-opti.patchapplication/octet-stream; name=v60-0001-Introduce-a-max_conflict_retention_duration-opti.patchDownload
From 4f221131a16d6c70938c790ff2329e19aa2bb038 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v60 1/2] Introduce a 'max_conflict_retention_duration' option
to subscriptions.
This commit introduces a subscription option max_conflict_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it after confirming that the
replication slot has been dropped. An upcoming patch will include support for
automatic slot re-initialization 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 'dead_tuple_retention_active' 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_dead_tuples 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/monitoring.sgml | 13 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 41 +++++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 33 +++-
src/backend/replication/logical/launcher.c | 45 +++--
src/backend/replication/logical/tablesync.c | 22 +--
src/backend/replication/logical/worker.c | 177 +++++++++++++++++--
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 7 +
src/include/replication/worker_internal.h | 10 +-
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 185 +++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
src/test/subscription/t/035_conflicts.pl | 10 +-
20 files changed, 480 insertions(+), 136 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..e9d519bf382 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which information useful for conflict
+ detection is retained by this apply worker does not exceed
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..f2c2e147472 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..6121c86561e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration that the apply worker, according to this subscription,
+ is allowed to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention, you can disable
+ <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that Overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..9da33cbcc28 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxconflretention = subform->maxconflretention;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..0c803ca6c43 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
FROM pg_subscription su
LEFT JOIN pg_stat_get_subscription(NULL) st
ON (st.subid = su.oid);
@@ -1389,8 +1390,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, maxconflretention, 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 faa3650d287..5d423d7be7a 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxconflretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -693,6 +707,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_maxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1235,7 +1251,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1436,6 +1454,13 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_maxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ replaces[Anum_pg_subscription_maxconflretention - 1] = true;
+ }
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..f1dfb51ccf8 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -998,7 +1000,7 @@ ApplyLauncherShmemInit(void)
LogicalRepWorker *worker = &LogicalRepCtx->workers[slot];
memset(worker, 0, sizeof(LogicalRepWorker));
- SpinLockInit(&worker->relmutex);
+ SpinLockInit(&worker->mutex);
}
}
}
@@ -1320,13 +1322,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention, the
+ * new xmin will be set to InvalidTransactionId. We then update
+ * slot.xmin accordingly to permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1374,11 +1381,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
*/
Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
+ SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockRelease(&worker->mutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1414,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1530,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;
@@ -1595,6 +1607,15 @@ 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))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3356bc84ee..1ab5496f63f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -293,7 +293,7 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
@@ -305,7 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
MyLogicalRepWorker->relstate_lsn = current_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* UpdateSubscriptionRelState must be called within a transaction.
@@ -390,7 +390,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
finish_sync_worker();
}
else
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
}
/*
@@ -534,7 +534,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
if (syncworker)
{
/* Found one, update our copy of its state */
- SpinLockAcquire(&syncworker->relmutex);
+ SpinLockAcquire(&syncworker->mutex);
rstate->state = syncworker->relstate;
rstate->lsn = syncworker->relstate_lsn;
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -547,7 +547,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
syncworker->relstate_lsn =
Max(syncworker->relstate_lsn, current_lsn);
}
- SpinLockRelease(&syncworker->relmutex);
+ SpinLockRelease(&syncworker->mutex);
/* If we told worker to catch up, wait for it. */
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -1342,10 +1342,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
must_use_password = MySubscription->passwordrequired &&
!MySubscription->ownersuperuser;
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = relstate;
MyLogicalRepWorker->relstate_lsn = relstate_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* If synchronization is already done or no longer necessary, exit now
@@ -1428,10 +1428,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
goto copy_table_done;
}
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/* Update the state and make it visible to others. */
StartTransactionCommand();
@@ -1586,10 +1586,10 @@ copy_table_done:
/*
* We are done with the initial data synchronization, update the state.
*/
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCWAIT;
MyLogicalRepWorker->relstate_lsn = *origin_startpos;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* Finally, wait until the leader apply worker tells us to catch up and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0fdc5de57ba..5269cc980f2 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3225,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool stop_retention;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3234,42 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ */
+ if (am_leader_apply_worker())
+ {
+ stop_retention =
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->mutex);
+ stop_retention = !TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->mutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (stop_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3296,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4160,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4376,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4468,6 +4523,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4611,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4583,9 +4666,9 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rdt_data->remote_lsn),
@@ -4594,12 +4677,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4699,66 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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->oldest_nonremovable_xid
+ * to InvalidTransactionId, notify the launcher to set the slot.xmin to
+ * InvalidTransactionId as well, and return true. Return false otherwise.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxconflretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxconflretention +
+ rdt_data->table_sync_wait_time))
+ 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.",
+ MySubscription->maxconflretention));
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4642,6 +4791,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
}
else
{
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index f3a353a61a5..db7908bb3d0 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5024,6 +5024,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_maxconflretention;
int i,
ntups;
@@ -5103,10 +5104,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.maxconflretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS maxconflretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5141,6 +5149,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_maxconflretention = PQfnumber(res, "maxconflretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5176,6 +5185,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].maxconflretention =
+ atoi(PQgetvalue(res, i, i_maxconflretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5437,6 +5448,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->maxconflretention)
+ appendPQExpBuffer(query, ", maxconflretention = %d", subinfo->maxconflretention);
+
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 dde85ed156c..6c302177f4a 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int maxconflretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..f2940fbf7a4 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", maxconflretention AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 1f2ca946fc5..6c5f2c90dfc 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..e7ed9dafae4 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,10 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +140,9 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..9c0c2b8050c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -64,7 +64,12 @@ typedef struct LogicalRepWorker
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
- slock_t relmutex;
+
+ /*
+ * Spinlock used to protect table synchronization information and the
+ * oldest_nonremovable_xid.
+ */
+ slock_t mutex;
/*
* Used to create the changes and subxact files for the streaming
@@ -94,6 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..7dbfacab37f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,35 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+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 dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..9b2c489adaf 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Mon, Aug 11, 2025 at 2:40 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I agree. So, following the above points and some off-list discussions, I have
revised the option to be a subscription option in the V60 version.
Thank You for the patches. Tried to test the new sub-level parameter,
have few comments:
1)
Let's say commit on pub is taking time and worker has stopped
retention and meanwhile we alter max_conflict_retention_duration=0,
then the expectation is immediately worker should resume retention.
But it does not happen, it does not restart conflict-retention until
the pub's commit is finished. The 'dead_tuple_retention_active'
remains 'f' till then.
postgres=# select subname, subretaindeadtuples, maxconflretention from
pg_subscription order by subname;
subname | subretaindeadtuples | maxconflretention
---------+---------------------+-------------------
sub1 | t | 0
postgres=# select subname, worker_type, dead_tuple_retention_active
from pg_stat_subscription order by subname;
subname | worker_type | dead_tuple_retention_active
---------+-------------+-----------------------------
sub1 | apply | f
I think we shall reset 'stop_conflict_info_retention' flag in
should_stop_conflict_info_retention() if maxconflretention is 0 and
the flag is originally true.
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher
CREATE SUBSCRIPTION
Shall we give notice that max_conflict_retention_duration is ignored
as retain_dead_tuples is false.
3)
When worker stops retention, it gives message:
LOG: logical replication worker for subscription "sub1" will stop
retaining the information for detecting conflicts
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 100 ms.
Will it be more informative if we mention the parameter name
'max_conflict_retention_duration' either in DETAIL or in additional
HINT, as then the user can easily map this behaviour to the parameter
configured.
thanks
Shveta
On Tue, Aug 12, 2025 at 2:06 PM shveta malik <shveta.malik@gmail.com> wrote:
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher
CREATE SUBSCRIPTIONShall we give notice that max_conflict_retention_duration is ignored
as retain_dead_tuples is false.
How about disallowing this combination?
--
With Regards,
Amit Kapila.
On Tue, Aug 12, 2025 at 2:21 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Aug 12, 2025 at 2:06 PM shveta malik <shveta.malik@gmail.com> wrote:
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher
CREATE SUBSCRIPTIONShall we give notice that max_conflict_retention_duration is ignored
as retain_dead_tuples is false.How about disallowing this combination?
+1 to disallow that.
--
Regards,
Dilip Kumar
Google
On Tuesday, August 12, 2025 5:01 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
Hi,
On Tue, Aug 12, 2025 at 2:21 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Tue, Aug 12, 2025 at 2:06 PM shveta malik <shveta.malik@gmail.com>
wrote:
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher CREATE
SUBSCRIPTIONShall we give notice that max_conflict_retention_duration is ignored
as retain_dead_tuples is false.How about disallowing this combination?
+1 to disallow that.
I think disallowing this case may not suffice, as users could initially set
(retain_dead_tuples=on, max_conflict_retention_duration=100) but later disable
retain_dead_tuples. This would result in the same state as
(retain_dead_tuples=off, max_conflict_retention_duration=100) unless we disallow
disabling rdt in this case as well.
So, do you think we should disallow both cases, or we only disallow setting
max_conflict_retention_duration for disabled retain_dead_tuples and give NOTICE
in another case ?
I personally prefer a consistent behavior, e.g., either we allow both cases and
give NOTICEs, or we disallow both cases. This is because, if the goal here to
prevent potential misconfigurations by users, the scenario where a user disables
retain_dead_tuples might also be considered a similar misconfiguration. So, I'm
a bit concerned that the benefits of imposing a partial restriction may not
outweigh the risk of generating inconsistent behavior. (And I did not see
similar precedent).
What do you think ?
Best Regards,
Hou zj
On Tue, Aug 12, 2025 at 3:02 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, August 12, 2025 5:01 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
Hi,
On Tue, Aug 12, 2025 at 2:21 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Tue, Aug 12, 2025 at 2:06 PM shveta malik <shveta.malik@gmail.com>
wrote:
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher CREATE
SUBSCRIPTIONShall we give notice that max_conflict_retention_duration is ignored
as retain_dead_tuples is false.How about disallowing this combination?
+1 to disallow that.
I think disallowing this case may not suffice, as users could initially set
(retain_dead_tuples=on, max_conflict_retention_duration=100) but later disable
retain_dead_tuples. This would result in the same state as
(retain_dead_tuples=off, max_conflict_retention_duration=100) unless we disallow
disabling rdt in this case as well.So, do you think we should disallow both cases, or we only disallow setting
max_conflict_retention_duration for disabled retain_dead_tuples and give NOTICE
in another case ?I personally prefer a consistent behavior, e.g., either we allow both cases and
give NOTICEs, or we disallow both cases. This is because, if the goal here to
prevent potential misconfigurations by users, the scenario where a user disables
retain_dead_tuples might also be considered a similar misconfiguration. So, I'm
a bit concerned that the benefits of imposing a partial restriction may not
outweigh the risk of generating inconsistent behavior. (And I did not see
similar precedent).
I think setting 'retain_dead_tuples' to off should implicitly reset
'max_conflict_retention_duration', yeah but someone may argue if we
again set 'retain_dead_tuples' to on then we should set
'max_conflict_retention_duration' to its original value and that could
add an extra complexity.
So maybe after thinking again IMHO, we can just go with the current
behavior, like we allow max_conflict_retention_duration to be set to a
positive value without setting retain_dead_tuples and issue a NOTICE
(in both cases).
--
Regards,
Dilip Kumar
Google
On Tue, Aug 12, 2025 at 3:22 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, Aug 12, 2025 at 3:02 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, August 12, 2025 5:01 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
Hi,
On Tue, Aug 12, 2025 at 2:21 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Tue, Aug 12, 2025 at 2:06 PM shveta malik <shveta.malik@gmail.com>
wrote:
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher CREATE
SUBSCRIPTIONShall we give notice that max_conflict_retention_duration is ignored
as retain_dead_tuples is false.How about disallowing this combination?
+1 to disallow that.
I think disallowing this case may not suffice, as users could initially set
(retain_dead_tuples=on, max_conflict_retention_duration=100) but later disable
retain_dead_tuples. This would result in the same state as
(retain_dead_tuples=off, max_conflict_retention_duration=100) unless we disallow
disabling rdt in this case as well.So, do you think we should disallow both cases, or we only disallow setting
max_conflict_retention_duration for disabled retain_dead_tuples and give NOTICE
in another case ?I personally prefer a consistent behavior, e.g., either we allow both cases and
give NOTICEs, or we disallow both cases. This is because, if the goal here to
prevent potential misconfigurations by users, the scenario where a user disables
retain_dead_tuples might also be considered a similar misconfiguration. So, I'm
a bit concerned that the benefits of imposing a partial restriction may not
outweigh the risk of generating inconsistent behavior. (And I did not see
similar precedent).I think setting 'retain_dead_tuples' to off should implicitly reset
'max_conflict_retention_duration', yeah but someone may argue if we
again set 'retain_dead_tuples' to on then we should set
'max_conflict_retention_duration' to its original value and that could
add an extra complexity.
I have the exact same opinion here.
So maybe after thinking again IMHO, we can just go with the current
behavior, like we allow max_conflict_retention_duration to be set to a
positive value without setting retain_dead_tuples and issue a NOTICE
(in both cases).
I feel that displaying a NOTICE in all such commands where
'max_conflict_retention_duration' will have no meaning is good enough.
thanks
Shveta
On Mon, Aug 11, 2025 at 2:40 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I agree. So, following the above points and some off-list discussions, I have
revised the option to be a subscription option in the V60 version.
Thanks Hou-san for the patches.
I have tested the patches and are working as expected. I've a minor
comment for patch v60-0001.
@@ -4642,6 +4791,10 @@
adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool
new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+
The function comment needs an update as per above change. Currently, it says -
* The interval is reset to a minimum value of 100ms once there is some
* activity on the node.
But if MySubscription->maxconflretention is < 100ms, then it will be
set to maxconflretention value and not 100ms.
--
Thanks,
Nisha
On Tuesday, August 12, 2025 4:37 PM shveta malik <shveta.malik@gmail.com> wrote:
On Mon, Aug 11, 2025 at 2:40 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I agree. So, following the above points and some off-list discussions,
I have revised the option to be a subscription option in the V60 version.Thank You for the patches. Tried to test the new sub-level parameter, have few
comments:
Thanks for the comments.
1)
Let's say commit on pub is taking time and worker has stopped retention and
meanwhile we alter max_conflict_retention_duration=0,
then the expectation is immediately worker should resume retention.
But it does not happen, it does not restart conflict-retention until the pub's
commit is finished. The 'dead_tuple_retention_active'
remains 'f' till then.postgres=# select subname, subretaindeadtuples, maxconflretention from
pg_subscription order by subname; subname | subretaindeadtuples |
maxconflretention
---------+---------------------+-------------------
sub1 | t | 0postgres=# select subname, worker_type, dead_tuple_retention_active from
pg_stat_subscription order by subname; subname | worker_type |
dead_tuple_retention_active
---------+-------------+-----------------------------
sub1 | apply | fI think we shall reset 'stop_conflict_info_retention' flag in
should_stop_conflict_info_retention() if maxconflretention is 0 and the flag is
originally true.
Agreed. Thinking more, I think we can resume retention at more places, as in
each phase, we could have a possibility of resuming, so changed.
2)
postgres=# create subscription sub2 connection 'dbname=postgres
host=localhost user=shveta port=5433' publication pub2 WITH
(retain_dead_tuples = false, max_conflict_retention_duration=1000);
NOTICE: created replication slot "sub2" on publisher CREATE
SUBSCRIPTIONShall we give notice that max_conflict_retention_duration is ignored as
retain_dead_tuples is false.
Agreed. In addition to this command, I added the NOTICE for all
the cases when the max_conflict_retention_duration is ineffective as
discussed[1]/messages/by-id/CAJpy0uC81YgAmrA2ji2ZKbJK_qfvajuV6=yvcCWuFsQKqiED+A@mail.gmail.com.
3)
When worker stops retention, it gives message:LOG: logical replication worker for subscription "sub1" will stop retaining the
information for detecting conflicts
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 100 ms.Will it be more informative if we mention the parameter name
'max_conflict_retention_duration' either in DETAIL or in additional HINT, as
then the user can easily map this behaviour to the parameter configured.
Added.
Here is the V61 patch set which addressed above comments and the comment by Nisha[2]/messages/by-id/CABdArM7G1sSDDOEC-nmJRnJMCZoBsLqOMz08UotX_h_wqxHWCg@mail.gmail.com.
[1]: /messages/by-id/CAJpy0uC81YgAmrA2ji2ZKbJK_qfvajuV6=yvcCWuFsQKqiED+A@mail.gmail.com
[2]: /messages/by-id/CABdArM7G1sSDDOEC-nmJRnJMCZoBsLqOMz08UotX_h_wqxHWCg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v61-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v61-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From ce63de11056cd515e5f435ac408bf5a355ed0366 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 13 Aug 2025 11:33:24 +0800
Subject: [PATCH v61 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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/ref/create_subscription.sgml | 7 +-
src/backend/replication/logical/launcher.c | 83 ++++++++---
src/backend/replication/logical/worker.c | 151 ++++++++++++++++++---
src/include/replication/worker_internal.h | 7 +
4 files changed, 212 insertions(+), 36 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 4190c190ae8..73cc82d52a0 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -539,9 +539,12 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration exceeded the
<literal>max_conflict_retention_duration</literal> set within the
- corresponding subscription. To re-enable retention, you can disable
- <literal>retain_dead_tuples</literal> for all subscriptions and
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
re-enable it after confirming this replication slot has been dropped.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f1dfb51ccf8..b85acd95e6b 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -464,6 +467,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
@@ -1258,8 +1263,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1369,9 +1374,11 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool stop_retention;
Assert(worker != NULL);
@@ -1383,13 +1390,43 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->mutex);
/*
* Skip collecting oldest_nonremovable_xid for workers that have stopped
* conflict retention.
*/
+ if (stop_retention)
+ return;
+
+ /*
+ * Initialize slot.xmin as a apply worker resumes retention of information
+ * critical for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see get_candidate_xid).
+ */
if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->mutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->mutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
@@ -1451,23 +1488,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1487,6 +1516,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index afa3f96ea3c..8f1bf276384 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -418,6 +418,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -560,6 +564,7 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4376,10 +4381,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4417,6 +4418,41 @@ get_candidate_xid(RetainDeadTuplesData *rdt_data)
TransactionId oldest_running_xid;
TimestampTz now;
+ /*
+ * If retention has been stopped but max_conflict_retention_duration is now
+ * set to 0, resume retention immediately.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention &&
+ !MySubscription->maxconflretention)
+ resume_conflict_info_retention(rdt_data);
+
+ /*
+ * No need to advance if the apply worker has resumed retention but the
+ * launcher has not yet initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ rdt_data->wait_for_initial_xid = false;
+ }
+
/*
* Use last_recv_time when applying changes in the loop to avoid
* unnecessary system time retrieval. If last_recv_time is not available,
@@ -4523,6 +4559,23 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * If retention has been stopped but max_conflict_retention_duration is now
+ * set to 0, resume retention immediately without proceeding to next phases.
+ *
+ * Do not check this condition before receiving the publisher message, as
+ * doing so may result in the message being processed after the data has
+ * been reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention &&
+ !MySubscription->maxconflretention)
+ {
+ resume_conflict_info_retention(rdt_data);
+ return;
+ }
+
/*
* Stop retaining conflict information if required (See
* should_stop_conflict_info_retention() for details).
@@ -4575,6 +4628,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
Assert(!XLogRecPtrIsInvalid(rdt_data->remote_lsn) &&
TransactionIdIsValid(rdt_data->candidate_xid));
+ /*
+ * If retention has been stopped but max_conflict_retention_duration is now
+ * set to 0, resume retention immediately without waiting for the local
+ * flush.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention &&
+ !MySubscription->maxconflretention)
+ {
+ resume_conflict_info_retention(rdt_data);
+ return;
+ }
+
/*
* We expect the publisher and subscriber clocks to be in sync using time
* sync service like NTP. Otherwise, we will advance this worker's
@@ -4661,6 +4726,17 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ resume_conflict_info_retention(rdt_data);
+ 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
@@ -4710,9 +4786,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* to InvalidTransactionId, notify the launcher to set the slot.xmin to
* InvalidTransactionId as well, and return true. Return false otherwise.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4743,24 +4818,65 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
+ /*
+ * 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.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+ }
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
+}
+
+/*
+ * Resume the retention if conflict info retention was previously stopped due to
+ * a timeout, and the time required to advance the non-removable transaction ID
+ * has now decreased to within acceptable limits.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ Assert(MyLogicalRepWorker->stop_conflict_info_retention);
+
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" will stop retaining conflict information",
+ errmsg("logical replication worker for subscription \"%s\" resumes retaining the information for detecting conflicts",
MySubscription->name),
- errdetail("The time spent advancing the non-removable transaction ID has exceeded the maximum limit of %u ms.",
- MySubscription->maxconflretention),
- errhint("You might need to increase \"%s\".",
- "max_conflict_retention_duration"));
+ errdetail("The time spent applying changes up to LSN %X/%X is now within the maximum limit of %u ms.",
+ LSN_FORMAT_ARGS(rdt_data->remote_lsn),
+ MySubscription->maxconflretention));
+
+ /*
+ * The next step is to wait for the launcher to initialize the
+ * oldest_nonremovable_xid.
+ */
+ rdt_data->wait_for_initial_xid = true;
SpinLockAcquire(&MyLogicalRepWorker->mutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->mutex);
- /* Notify launcher to update the conflict slot */
- ApplyLauncherWakeup();
-
reset_retention_data_fields(rdt_data);
- return true;
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
}
/*
@@ -5611,6 +5727,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ !MyLogicalRepWorker->stop_conflict_info_retention &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9c0c2b8050c..32a1cbb6528 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -105,6 +105,13 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining information
+ * useful for conflict detection. This is used only when
+ * retain_dead_tuples is enabled.
+ */
+ bool stop_conflict_info_retention;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.50.1.windows.1
v61-0001-Introduce-a-max_conflict_retention_duration-opti.patchapplication/octet-stream; name=v61-0001-Introduce-a-max_conflict_retention_duration-opti.patchDownload
From a8fc7b1876055b4483b1a6070e08d115f6e9e230 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v61 1/2] Introduce a 'max_conflict_retention_duration' option
to subscriptions.
This commit introduces a subscription option max_conflict_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it after confirming that the
replication slot has been dropped. An upcoming patch will include support for
automatic slot re-initialization 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 'dead_tuple_retention_active' 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_dead_tuples 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/monitoring.sgml | 13 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 41 +++++
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 62 ++++++-
src/backend/replication/logical/launcher.c | 45 +++--
src/backend/replication/logical/tablesync.c | 22 +--
src/backend/replication/logical/worker.c | 183 +++++++++++++++++--
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 7 +
src/include/replication/worker_internal.h | 10 +-
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 186 +++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
src/test/subscription/t/035_conflicts.pl | 10 +-
20 files changed, 514 insertions(+), 138 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..e9d519bf382 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,19 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which information useful for conflict
+ detection is retained by this apply worker does not exceed
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..f2c2e147472 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..4190c190ae8 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration that the apply worker, according to this subscription,
+ is allowed to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention, you can disable
+ <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..9da33cbcc28 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxconflretention = subform->maxconflretention;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..0c803ca6c43 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
FROM pg_subscription su
LEFT JOIN pg_stat_get_subscription(NULL) st
ON (st.subid = su.oid);
@@ -1389,8 +1390,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, maxconflretention, 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 faa3650d287..111151756fb 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxconflretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -112,6 +114,7 @@ static void check_publications_origin(WalReceiverConn *wrconn,
Oid *subrel_local_oids, int subrel_count,
char *subname);
static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
+static void notify_ineffective_max_conflict_retention(void);
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);
@@ -169,6 +172,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +328,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +594,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -651,6 +666,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
if (opts.retaindeadtuples)
CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /* Notify that max_conflict_retention_duration is ineffective */
+ else if (opts.maxconflretention)
+ notify_ineffective_max_conflict_retention();
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -693,6 +712,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_maxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1196,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_failover = false;
bool update_two_phase = false;
bool check_pub_rdt = false;
+ bool ineffective_maxconflretention = false;
bool retain_dead_tuples;
char *origin;
Subscription *sub;
@@ -1235,7 +1257,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1434,6 +1458,19 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
check_pub_rdt = opts.retaindeadtuples;
retain_dead_tuples = opts.retaindeadtuples;
+
+ ineffective_maxconflretention = (!opts.retaindeadtuples &&
+ sub->maxconflretention);
+ }
+
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_maxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ replaces[Anum_pg_subscription_maxconflretention - 1] = true;
+
+ ineffective_maxconflretention = (!retain_dead_tuples &&
+ opts.maxconflretention);
}
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
@@ -1453,6 +1490,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
origin = opts.origin;
}
+ if (ineffective_maxconflretention)
+ notify_ineffective_max_conflict_retention();
+
update_tuple = true;
break;
}
@@ -2500,6 +2540,20 @@ CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
"retain_dead_tuples") : 0);
}
+/*
+ * Report a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_conflict_retention_duration causes no harm,
+ * even when it is ineffective.
+ */
+static void
+notify_ineffective_max_conflict_retention(void)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled"));
+}
+
/*
* 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 37377f7eb63..f1dfb51ccf8 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -998,7 +1000,7 @@ ApplyLauncherShmemInit(void)
LogicalRepWorker *worker = &LogicalRepCtx->workers[slot];
memset(worker, 0, sizeof(LogicalRepWorker));
- SpinLockInit(&worker->relmutex);
+ SpinLockInit(&worker->mutex);
}
}
}
@@ -1320,13 +1322,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention, the
+ * new xmin will be set to InvalidTransactionId. We then update
+ * slot.xmin accordingly to permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1374,11 +1381,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
*/
Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
+ SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockRelease(&worker->mutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1414,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1530,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;
@@ -1595,6 +1607,15 @@ 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))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3356bc84ee..1ab5496f63f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -293,7 +293,7 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
@@ -305,7 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
MyLogicalRepWorker->relstate_lsn = current_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* UpdateSubscriptionRelState must be called within a transaction.
@@ -390,7 +390,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
finish_sync_worker();
}
else
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
}
/*
@@ -534,7 +534,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
if (syncworker)
{
/* Found one, update our copy of its state */
- SpinLockAcquire(&syncworker->relmutex);
+ SpinLockAcquire(&syncworker->mutex);
rstate->state = syncworker->relstate;
rstate->lsn = syncworker->relstate_lsn;
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -547,7 +547,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
syncworker->relstate_lsn =
Max(syncworker->relstate_lsn, current_lsn);
}
- SpinLockRelease(&syncworker->relmutex);
+ SpinLockRelease(&syncworker->mutex);
/* If we told worker to catch up, wait for it. */
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -1342,10 +1342,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
must_use_password = MySubscription->passwordrequired &&
!MySubscription->ownersuperuser;
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = relstate;
MyLogicalRepWorker->relstate_lsn = relstate_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* If synchronization is already done or no longer necessary, exit now
@@ -1428,10 +1428,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
goto copy_table_done;
}
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/* Update the state and make it visible to others. */
StartTransactionCommand();
@@ -1586,10 +1586,10 @@ copy_table_done:
/*
* We are done with the initial data synchronization, update the state.
*/
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCWAIT;
MyLogicalRepWorker->relstate_lsn = *origin_startpos;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* Finally, wait until the leader apply worker tells us to catch up and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 0fdc5de57ba..afa3f96ea3c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3225,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool stop_retention;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3234,42 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ */
+ if (am_leader_apply_worker())
+ {
+ stop_retention =
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->mutex);
+ stop_retention = !TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->mutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (stop_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3296,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4160,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4376,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4468,6 +4523,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4611,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4583,9 +4666,9 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rdt_data->remote_lsn),
@@ -4594,12 +4677,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4699,68 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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->oldest_nonremovable_xid
+ * to InvalidTransactionId, notify the launcher to set the slot.xmin to
+ * InvalidTransactionId as well, and return true. Return false otherwise.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxconflretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxconflretention +
+ rdt_data->table_sync_wait_time))
+ 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.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4621,8 +4772,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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.
+ * The interval is reset to the lesser of 100ms and
+ * max_conflict_retention_duration once there is some activities 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.
@@ -4642,6 +4793,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
}
else
{
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..b993071eb78 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_maxconflretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.maxconflretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS maxconflretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_maxconflretention = PQfnumber(res, "maxconflretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].maxconflretention =
+ atoi(PQgetvalue(res, i, i_maxconflretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->maxconflretention)
+ appendPQExpBuffer(query, ", maxconflretention = %d", subinfo->maxconflretention);
+
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 dde85ed156c..6c302177f4a 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int maxconflretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..f2940fbf7a4 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", maxconflretention AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..2d7016fe717 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..e7ed9dafae4 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,10 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +140,9 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..9c0c2b8050c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -64,7 +64,12 @@ typedef struct LogicalRepWorker
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
- slock_t relmutex;
+
+ /*
+ * Spinlock used to protect table synchronization information and the
+ * oldest_nonremovable_xid.
+ */
+ slock_t mutex;
/*
* Used to create the changes and subxact files for the streaming
@@ -94,6 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..16b173a7ea0 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+NOTICE: max_conflict_retention_duration has no effect when retain_dead_tuples 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 dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..9b2c489adaf 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Wed, Aug 13, 2025 at 10:41 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V61 patch set which addressed above comments and the comment by Nisha[2].
Thank You for the patch. I tested the patch, please find a few comments:
1)
Now when it stops-retention and later resumes it due to the fact that
max_duration is meanwhile altered to 0, I get log:
LOG: logical replication worker for subscription "sub1" resumes
retaining the information for detecting conflicts
DETAIL: The time spent applying changes up to LSN 0/17DD728 is now
within the maximum limit of 0 ms.
I did not get which lsn it is pointing to? Is it some dangling lsn
from when it was retaining info? Also the msg looks odd, when it says
'is now within the maximum limit of 0 ms.'
2)
While stopping the message is:
LOG: logical replication worker for subscription "sub1" will stop
retaining conflict information
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 1000 ms.
And while resuming:
logical replication worker for subscription "sub1" resumes retaining
the information for detecting conflicts
----------
We can make both similar. Both can have 'retaining the information for
detecting conflicts' instead of 'conflict information' in first one.
3)
I believe the tenses should also be updated. When stopping, we can say:
Logical replication worker for subscription "sub1" has stopped...
This is appropriate because it has already stopped by pre-setting
oldest_nonremovable_xid to Invalid.
When resuming, we can say:
Logical replication worker for subscription "sub1" will resume...
This is because it will begin resuming from the next cycle onward,
specifically after the launcher sets its oldest_xid.
4)
For the DETAIL part of resume and stop messages, how about these:
The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.
thanks
Shveta
On Wed, Aug 13, 2025 at 4:15 PM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 13, 2025 at 10:41 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V61 patch set which addressed above comments and the comment by Nisha[2].
Thank You for the patch. I tested the patch, please find a few comments:
1)
Now when it stops-retention and later resumes it due to the fact that
max_duration is meanwhile altered to 0, I get log:LOG: logical replication worker for subscription "sub1" resumes
retaining the information for detecting conflicts
DETAIL: The time spent applying changes up to LSN 0/17DD728 is now
within the maximum limit of 0 ms.I did not get which lsn it is pointing to? Is it some dangling lsn
from when it was retaining info? Also the msg looks odd, when it says
'is now within the maximum limit of 0 ms.'2)
While stopping the message is:
LOG: logical replication worker for subscription "sub1" will stop
retaining conflict information
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 1000 ms.And while resuming:
logical replication worker for subscription "sub1" resumes retaining
the information for detecting conflicts
----------We can make both similar. Both can have 'retaining the information for
detecting conflicts' instead of 'conflict information' in first one.
Can we try to keep it slightly short by saying "retaining conflict
detection info"?
4)
For the DETAIL part of resume and stop messages, how about these:The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.
Similar to the previous point, will it be better to keep it short by
using "conflict detection info", for example, it will lead to message
like "The retention duration for conflict detection info is now
indefinite."?
--
With Regards,
Amit Kapila.
On Wed, Aug 13, 2025 at 5:42 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
4)
For the DETAIL part of resume and stop messages, how about these:The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.Similar to the previous point, will it be better to keep it short by
using "conflict detection info", for example, it will lead to message
like "The retention duration for conflict detection info is now
indefinite."?
Works for me as I understand the context. But could it be confusing
for users? Could it be inferred as info about conflicts rather than
info used while detecting conflicts?
thanks
Shveta
On Wed, Aug 13, 2025 at 4:15 PM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 13, 2025 at 10:41 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V61 patch set which addressed above comments and the comment by Nisha[2].
Thank You for the patch. I tested the patch, please find a few comments:
1)
Now when it stops-retention and later resumes it due to the fact that
max_duration is meanwhile altered to 0, I get log:LOG: logical replication worker for subscription "sub1" resumes
retaining the information for detecting conflicts
DETAIL: The time spent applying changes up to LSN 0/17DD728 is now
within the maximum limit of 0 ms.I did not get which lsn it is pointing to? Is it some dangling lsn
from when it was retaining info? Also the msg looks odd, when it says
'is now within the maximum limit of 0 ms.'2)
While stopping the message is:
LOG: logical replication worker for subscription "sub1" will stop
retaining conflict information
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 1000 ms.And while resuming:
logical replication worker for subscription "sub1" resumes retaining
the information for detecting conflicts
----------We can make both similar. Both can have 'retaining the information for
detecting conflicts' instead of 'conflict information' in first one.3)
I believe the tenses should also be updated. When stopping, we can say:Logical replication worker for subscription "sub1" has stopped...
This is appropriate because it has already stopped by pre-setting
oldest_nonremovable_xid to Invalid.When resuming, we can say:
Logical replication worker for subscription "sub1" will resume...This is because it will begin resuming from the next cycle onward,
specifically after the launcher sets its oldest_xid.4)
For the DETAIL part of resume and stop messages, how about these:The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.
5)
Say there 2-3 subs, all have stopped-retention and the slot is set to
have invalid xmin; now if I create a new sub, it will start with
stopped-flag set to true due to the fact that slot has invalid xmin to
begin with. But then immediately, it will dump a resume message. It
looks odd, as at first, it has not even stopped, as it is a new sub.
Is there anything we can do to improve this situation?
Logs:
2025-08-13 15:13:01.197 IST [61926] LOG: logical replication apply
worker for subscription "sub4" has started
2025-08-13 15:13:01.482 IST [61926] LOG: logical replication worker
for subscription "sub4" resumes retaining the information for
detecting conflicts
2025-08-13 15:13:01.482 IST [61926] DETAIL: The time spent applying
changes up to LSN 0/17C9348 is now within the maximum limit of 50000
ms.
thanks
Shveta
Few more comments:
1)
src/sgml/monitoring.sgml:
+ <para>
+ True if <link
linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the duration for which information useful for conflict
+ detection is retained by this apply worker does not exceed
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>;
NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
How about:
True for apply workers if retain_dead_tuples is enabled and the
conflict detection information retention time is within
max_conflict_retention_duration; NULL for parallel apply and table
synchronization workers.
(I have used 'conflict detection information' here, as suggested by
Amit in another email. But if we plan to stick to another version in
LOG message, please do the same here as well)
2)
create_subscription.sgml: retain_dead_tuples:
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named
<quote><literal>pg_conflict_detection</literal></quote>
created on the subscriber to prevent the information for detecting
conflicts from being removed.
'created on the subscriber' --> 'is created on the subscriber'.
'is' missing. It belongs to the committed patch, but shall we change
it here?
3)
+ Maximum duration that the apply worker, according to this
subscription,
+ is allowed to retain the information useful for conflict
detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions.
Shall we say:
'Maximum duration for which this subscription's apply worker is
allowed to retain...'.
4)
if (MyLogicalRepWorker->stop_conflict_info_retention &&
!MySubscription->maxconflretention)
{
resume_conflict_info_retention(rdt_data);
return;
}
The above logic is there in all phases beginning except 'request
status'. Do you think we can shift it to
maybe_advance_nonremovable_xid() before we call
process_rdt_phase_transition()? We do not need to 'return' there, it
can proceed with get_candidate_xid immediately in that call itself
once it does reset in resume-call.
thanks
Shveta
On Thu, Aug 14, 2025 at 9:02 AM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 13, 2025 at 5:42 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
4)
For the DETAIL part of resume and stop messages, how about these:The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.Similar to the previous point, will it be better to keep it short by
using "conflict detection info", for example, it will lead to message
like "The retention duration for conflict detection info is now
indefinite."?Works for me as I understand the context. But could it be confusing
for users? Could it be inferred as info about conflicts rather than
info used while detecting conflicts?
Fair point. We can retain your proposed message.
--
With Regards,
Amit Kapila.
On Thursday, August 14, 2025 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 13, 2025 at 4:15 PM shveta malik <shveta.malik@gmail.com>
wrote:On Wed, Aug 13, 2025 at 10:41 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V61 patch set which addressed above comments and the
comment by Nisha[2].
Thank You for the patch. I tested the patch, please find a few comments:
1)
Now when it stops-retention and later resumes it due to the fact that
max_duration is meanwhile altered to 0, I get log:LOG: logical replication worker for subscription "sub1" resumes
retaining the information for detecting conflicts
DETAIL: The time spent applying changes up to LSN 0/17DD728 is now
within the maximum limit of 0 ms.I did not get which lsn it is pointing to? Is it some dangling lsn
from when it was retaining info? Also the msg looks odd, when it says
'is now within the maximum limit of 0 ms.'2)
While stopping the message is:
LOG: logical replication worker for subscription "sub1" will stop
retaining conflict information
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 1000 ms.And while resuming:
logical replication worker for subscription "sub1" resumes retaining
the information for detecting conflicts
----------We can make both similar. Both can have 'retaining the information for
detecting conflicts' instead of 'conflict information' in first one.3)
I believe the tenses should also be updated. When stopping, we can say:Logical replication worker for subscription "sub1" has stopped...
This is appropriate because it has already stopped by pre-setting
oldest_nonremovable_xid to Invalid.When resuming, we can say:
Logical replication worker for subscription "sub1" will resume...This is because it will begin resuming from the next cycle onward,
specifically after the launcher sets its oldest_xid.4)
For the DETAIL part of resume and stop messages, how about these:The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.
Thanks for the comments, I have adjusted the log messages
according to the suggestions.
5)
Say there 2-3 subs, all have stopped-retention and the slot is set to have invalid
xmin; now if I create a new sub, it will start with stopped-flag set to true due to
the fact that slot has invalid xmin to begin with. But then immediately, it will
dump a resume message. It looks odd, as at first, it has not even stopped, as it
is a new sub.
Is there anything we can do to improve this situation?
I changed the logic to recovery the slot immediately on starting a new worker
that has retain_dead_tuples enabled.
Here is the V62 patch set which addressed above comments and [1]/messages/by-id/CAJpy0uBW8G2RNY=Jjxzr_ootQ2MTxPQG98hz=-wdJzn86yapVg@mail.gmail.com.
[1]: /messages/by-id/CAJpy0uBW8G2RNY=Jjxzr_ootQ2MTxPQG98hz=-wdJzn86yapVg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v62-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v62-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 49a0902648e609771502d17c6dde178331128a2f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 13 Aug 2025 11:33:24 +0800
Subject: [PATCH v62 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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/ref/create_subscription.sgml | 11 +-
src/backend/replication/logical/launcher.c | 104 +++++++++++++----
src/backend/replication/logical/worker.c | 129 ++++++++++++++++++---
src/include/replication/worker_internal.h | 7 ++
4 files changed, 208 insertions(+), 43 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 708284dafdf..644d4fe5aaa 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -533,26 +533,31 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
that the information is retained until it is no longer needed for
detection purposes. This value is taken as milliseconds.
</para>
+
<para>
The information useful for conflict detection is no longer retained if
all apply workers associated with the subscriptions, where
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration exceeded the
<literal>max_conflict_retention_duration</literal> set within the
- corresponding subscription. To re-enable retention, you can disable
- <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ corresponding subscription. The retention will be automatically resumed
+ once at least one apply worker confirms that the retention duration is
+ within the specified limit, or if a new apply worker with
+ retain_dead_tuples enabled is started.
</para>
+
<para>
Note that overall retention will not stop if other subscriptions
specify a greater value and have not exceeded it, or if they set this
option to 0.
</para>
+
<para>
This option is effective only when
<literal>retain_conflict_info</literal> is enabled and the apply
worker associated with the subscription is active.
</para>
+
<warning>
<para>
Note that setting a non-zero value for this option could lead to
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index f1dfb51ccf8..ae70e5441cb 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -464,6 +467,8 @@ retry:
worker->stream_fileset = NULL;
worker->leader_pid = is_parallel_apply_worker ? MyProcPid : InvalidPid;
worker->parallel_apply = is_parallel_apply_worker;
+ worker->stop_conflict_info_retention = (retain_dead_tuples &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
@@ -1258,21 +1263,30 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
}
- /*
- * Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
- * nonremovable xid.
- */
if (sub->retaindeadtuples)
+ {
+ /*
+ * Initialize slot.xmin for a new apply worker that requests the
+ * retention of information useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+
+ /*
+ * Can't advance xmin of the slot unless all the workers
+ * corresponding to subscriptions with retain_dead_tuples are
+ * running, disabling the further computation of the minimum
+ * nonremovable xid.
+ */
can_advance_xmin = false;
+ }
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1369,9 +1383,11 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool stop_retention;
Assert(worker != NULL);
@@ -1383,13 +1399,43 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ stop_retention = worker->stop_conflict_info_retention;
SpinLockRelease(&worker->mutex);
/*
* Skip collecting oldest_nonremovable_xid for workers that have stopped
* conflict retention.
*/
+ if (stop_retention)
+ return;
+
+ /*
+ * Initialize slot.xmin as a apply worker resumes retention of information
+ * useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see get_candidate_xid).
+ */
if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->mutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->mutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
@@ -1451,23 +1497,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1487,6 +1525,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index b4d2ab41a2d..8316db1caf7 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -418,6 +418,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -560,6 +564,7 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4354,6 +4359,21 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * If retention has been stopped but max_conflict_retention_duration is now
+ * set to 0, resume retention immediately.
+ *
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been reset,
+ * potentially causing it to be mistakenly identified as a new message. This
+ * could lead to the premature advancement of oldest_nonremovable_xid.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention &&
+ !MySubscription->maxconflretention &&
+ !(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received))
+ resume_conflict_info_retention(rdt_data);
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4376,10 +4396,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4417,6 +4433,33 @@ get_candidate_xid(RetainDeadTuplesData *rdt_data)
TransactionId oldest_running_xid;
TimestampTz now;
+ /*
+ * No need to advance if the apply worker has resumed retention but the
+ * launcher has not yet initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ rdt_data->wait_for_initial_xid = false;
+ }
+
/*
* Use last_recv_time when applying changes in the loop to avoid
* unnecessary system time retrieval. If last_recv_time is not available,
@@ -4661,6 +4704,17 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (MyLogicalRepWorker->stop_conflict_info_retention)
+ {
+ resume_conflict_info_retention(rdt_data);
+ 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
@@ -4710,9 +4764,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* to InvalidTransactionId, notify the launcher to set the slot.xmin to
* InvalidTransactionId as well, and return true. Return false otherwise.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4743,24 +4796,66 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
+ /*
+ * 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\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = true;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+ }
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
+}
+
+/*
+ * Resume the retention if conflict info retention was previously stopped due to
+ * a timeout, and the time required to advance the non-removable transaction ID
+ * has now decreased to within acceptable limits.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ Assert(MyLogicalRepWorker->stop_conflict_info_retention);
+
ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
MySubscription->name),
- errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
- MySubscription->maxconflretention),
- errhint("You might need to increase \"%s\".",
- "max_conflict_retention_duration"));
+ MySubscription->maxconflretention
+ ? errdetail("The retention duration for information used in conflict detection is now within the acceptable limit of %u ms.",
+ MySubscription->maxconflretention)
+ : errdetail("The retention duration for information used in conflict detection is now indefinite."));
+
+ /*
+ * The next step is to wait for the launcher to initialize the
+ * oldest_nonremovable_xid.
+ */
+ rdt_data->wait_for_initial_xid = true;
SpinLockAcquire(&MyLogicalRepWorker->mutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ MyLogicalRepWorker->stop_conflict_info_retention = false;
SpinLockRelease(&MyLogicalRepWorker->mutex);
- /* Notify launcher to update the conflict slot */
- ApplyLauncherWakeup();
-
reset_retention_data_fields(rdt_data);
- return true;
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
}
/*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 9c0c2b8050c..32a1cbb6528 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -105,6 +105,13 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker has stopped retaining information
+ * useful for conflict detection. This is used only when
+ * retain_dead_tuples is enabled.
+ */
+ bool stop_conflict_info_retention;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.50.1.windows.1
v62-0001-Introduce-a-max_conflict_retention_duration-opti.patchapplication/octet-stream; name=v62-0001-Introduce-a-max_conflict_retention_duration-opti.patchDownload
From f83718838fd117cfe530cc6e95db3e29b52c714b Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v62 1/2] Introduce a 'max_conflict_retention_duration' option
to subscriptions.
This commit introduces a subscription option max_conflict_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it after confirming that the
replication slot has been dropped. An upcoming patch will include support for
automatic slot re-initialization 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 'dead_tuple_retention_active' 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_dead_tuples 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/monitoring.sgml | 12 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 ++++-
src/backend/catalog/pg_subscription.c | 1 +
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 66 ++++++-
src/backend/replication/logical/launcher.c | 45 +++--
src/backend/replication/logical/tablesync.c | 22 +--
src/backend/replication/logical/worker.c | 183 +++++++++++++++++--
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 8 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 7 +
src/include/replication/worker_internal.h | 10 +-
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 186 +++++++++++---------
src/test/regress/sql/subscription.sql | 16 ++
src/test/subscription/t/035_conflicts.pl | 10 +-
20 files changed, 518 insertions(+), 139 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..11bc4916b33 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..f2c2e147472 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..708284dafdf 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention, you can disable
+ <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..9da33cbcc28 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,7 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxconflretention = subform->maxconflretention;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..0c803ca6c43 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
FROM pg_subscription su
LEFT JOIN pg_stat_get_subscription(NULL) st
ON (st.subid = su.oid);
@@ -1389,8 +1390,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, maxconflretention, 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 faa3650d287..2e62a23b18b 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxconflretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -112,6 +114,7 @@ static void check_publications_origin(WalReceiverConn *wrconn,
Oid *subrel_local_oids, int subrel_count,
char *subname);
static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
+static void notify_ineffective_max_conflict_retention(bool update_maxconflretention);
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);
@@ -169,6 +172,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +328,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +594,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -651,6 +666,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
if (opts.retaindeadtuples)
CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /* Notify that max_conflict_retention_duration is ineffective */
+ else if (opts.maxconflretention)
+ notify_ineffective_max_conflict_retention(true);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -693,6 +712,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_maxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1196,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_failover = false;
bool update_two_phase = false;
bool check_pub_rdt = false;
+ bool ineffective_maxconflretention = false;
+ bool update_maxconflretention = false;
bool retain_dead_tuples;
char *origin;
Subscription *sub;
@@ -1235,7 +1258,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1434,6 +1459,20 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
check_pub_rdt = opts.retaindeadtuples;
retain_dead_tuples = opts.retaindeadtuples;
+
+ ineffective_maxconflretention = (!opts.retaindeadtuples &&
+ sub->maxconflretention);
+ }
+
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_maxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ replaces[Anum_pg_subscription_maxconflretention - 1] = true;
+
+ update_maxconflretention = true;
+ ineffective_maxconflretention = (!retain_dead_tuples &&
+ opts.maxconflretention);
}
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
@@ -1453,6 +1492,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
origin = opts.origin;
}
+ if (ineffective_maxconflretention)
+ notify_ineffective_max_conflict_retention(update_maxconflretention);
+
update_tuple = true;
break;
}
@@ -2500,6 +2542,22 @@ CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
"retain_dead_tuples") : 0);
}
+/*
+ * Report a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_conflict_retention_duration causes no harm,
+ * even when it is ineffective.
+ */
+static void
+notify_ineffective_max_conflict_retention(bool update_maxconflretention)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ update_maxconflretention
+ ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled")
+ : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective"));
+}
+
/*
* 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 37377f7eb63..f1dfb51ccf8 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -998,7 +1000,7 @@ ApplyLauncherShmemInit(void)
LogicalRepWorker *worker = &LogicalRepCtx->workers[slot];
memset(worker, 0, sizeof(LogicalRepWorker));
- SpinLockInit(&worker->relmutex);
+ SpinLockInit(&worker->mutex);
}
}
}
@@ -1320,13 +1322,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention, the
+ * new xmin will be set to InvalidTransactionId. We then update
+ * slot.xmin accordingly to permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1374,11 +1381,16 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
*/
Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
+ SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockRelease(&worker->mutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Skip collecting oldest_nonremovable_xid for workers that have stopped
+ * conflict retention.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1414,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1530,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;
@@ -1595,6 +1607,15 @@ 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))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3356bc84ee..1ab5496f63f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -293,7 +293,7 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
@@ -305,7 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
MyLogicalRepWorker->relstate_lsn = current_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* UpdateSubscriptionRelState must be called within a transaction.
@@ -390,7 +390,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
finish_sync_worker();
}
else
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
}
/*
@@ -534,7 +534,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
if (syncworker)
{
/* Found one, update our copy of its state */
- SpinLockAcquire(&syncworker->relmutex);
+ SpinLockAcquire(&syncworker->mutex);
rstate->state = syncworker->relstate;
rstate->lsn = syncworker->relstate_lsn;
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -547,7 +547,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
syncworker->relstate_lsn =
Max(syncworker->relstate_lsn, current_lsn);
}
- SpinLockRelease(&syncworker->relmutex);
+ SpinLockRelease(&syncworker->mutex);
/* If we told worker to catch up, wait for it. */
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -1342,10 +1342,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
must_use_password = MySubscription->passwordrequired &&
!MySubscription->ownersuperuser;
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = relstate;
MyLogicalRepWorker->relstate_lsn = relstate_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* If synchronization is already done or no longer necessary, exit now
@@ -1428,10 +1428,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
goto copy_table_done;
}
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/* Update the state and make it visible to others. */
StartTransactionCommand();
@@ -1586,10 +1586,10 @@ copy_table_done:
/*
* We are done with the initial data synchronization, update the state.
*/
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCWAIT;
MyLogicalRepWorker->relstate_lsn = *origin_startpos;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* Finally, wait until the leader apply worker tells us to catch up and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 59b6ae7719a..b4d2ab41a2d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -415,6 +415,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +558,8 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3225,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool stop_retention;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3234,42 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ */
+ if (am_leader_apply_worker())
+ {
+ stop_retention =
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->mutex);
+ stop_retention = !TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->mutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (stop_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3296,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4160,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4376,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4468,6 +4523,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4611,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4583,9 +4666,9 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%08X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rdt_data->remote_lsn),
@@ -4594,12 +4677,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_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. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
- */
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4699,68 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
- /* process the next phase */
- process_rdt_phase_transition(rdt_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->oldest_nonremovable_xid
+ * to InvalidTransactionId, notify the launcher to set the slot.xmin to
+ * InvalidTransactionId as well, and return true. Return false otherwise.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxconflretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxconflretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+
+ return true;
}
/*
@@ -4621,8 +4772,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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.
+ * The interval is reset to the lesser of 100ms and
+ * max_conflict_retention_duration once there is some activities 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.
@@ -4642,6 +4793,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
}
else
{
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..b993071eb78 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_maxconflretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.maxconflretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS maxconflretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_maxconflretention = PQfnumber(res, "maxconflretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].maxconflretention =
+ atoi(PQgetvalue(res, i, i_maxconflretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->maxconflretention)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->maxconflretention);
+
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 dde85ed156c..6c302177f4a 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int maxconflretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..f2940fbf7a4 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,16 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", maxconflretention AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..2d7016fe717 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..e7ed9dafae4 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,10 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +140,9 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be 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/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..9c0c2b8050c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -64,7 +64,12 @@ typedef struct LogicalRepWorker
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
- slock_t relmutex;
+
+ /*
+ * Spinlock used to protect table synchronization information and the
+ * oldest_nonremovable_xid.
+ */
+ slock_t mutex;
/*
* Used to create the changes and subxact files for the streaming
@@ -94,6 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..16b173a7ea0 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+NOTICE: max_conflict_retention_duration has no effect when retain_dead_tuples 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 dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..9b2c489adaf 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Thu, Aug 14, 2025 at 10:46 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thursday, August 14, 2025 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 13, 2025 at 4:15 PM shveta malik <shveta.malik@gmail.com>
wrote:On Wed, Aug 13, 2025 at 10:41 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V61 patch set which addressed above comments and the
comment by Nisha[2].
Thank You for the patch. I tested the patch, please find a few comments:
1)
Now when it stops-retention and later resumes it due to the fact that
max_duration is meanwhile altered to 0, I get log:LOG: logical replication worker for subscription "sub1" resumes
retaining the information for detecting conflicts
DETAIL: The time spent applying changes up to LSN 0/17DD728 is now
within the maximum limit of 0 ms.I did not get which lsn it is pointing to? Is it some dangling lsn
from when it was retaining info? Also the msg looks odd, when it says
'is now within the maximum limit of 0 ms.'2)
While stopping the message is:
LOG: logical replication worker for subscription "sub1" will stop
retaining conflict information
DETAIL: The time spent advancing the non-removable transaction ID has
exceeded the maximum limit of 1000 ms.And while resuming:
logical replication worker for subscription "sub1" resumes retaining
the information for detecting conflicts
----------We can make both similar. Both can have 'retaining the information for
detecting conflicts' instead of 'conflict information' in first one.3)
I believe the tenses should also be updated. When stopping, we can say:Logical replication worker for subscription "sub1" has stopped...
This is appropriate because it has already stopped by pre-setting
oldest_nonremovable_xid to Invalid.When resuming, we can say:
Logical replication worker for subscription "sub1" will resume...This is because it will begin resuming from the next cycle onward,
specifically after the launcher sets its oldest_xid.4)
For the DETAIL part of resume and stop messages, how about these:The retention duration for information used in conflict detection has
exceeded the limit of xx.
The retention duration for information used in conflict detection is
now within the acceptable limit of xx.
The retention duration for information used in conflict detection is
now indefinite.Thanks for the comments, I have adjusted the log messages
according to the suggestions.5)
Say there 2-3 subs, all have stopped-retention and the slot is set to have invalid
xmin; now if I create a new sub, it will start with stopped-flag set to true due to
the fact that slot has invalid xmin to begin with. But then immediately, it will
dump a resume message. It looks odd, as at first, it has not even stopped, as it
is a new sub.
Is there anything we can do to improve this situation?I changed the logic to recovery the slot immediately on starting a new worker
that has retain_dead_tuples enabled.Here is the V62 patch set which addressed above comments and [1].
Here are review comments on v62 patch:
Regarding the subscription-level option vs. GUC, I don't disagree with
the current approach.
For the record, while I agree that the subscription-level option is
more consistent with the existing retain_dead_tuples option and can
work for different requirements, my biggest concern is that if users
set different values to different subscriptions, they might think it
doesn't work as expected. This is because even if a subscription
disables retaining dead tuples due to max_conflict_retention_duration,
the database cluster doesn't stop retaining dead tuples unless all
other subscriptions disable it, meaning that the performance on that
database might not get recovered. I proposed the GUC parameter as I
thought it's less confusing from a user perspective. I'm not sure it's
sufficient to mention that in the documentation but I don't have a
better idea.
---
+ else if (IsSet(supported_opts,
SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname,
"max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts,
SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |=
SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
The new subscription parameter accepts only integers and takes it as
milliseconds, but I think it would be relatively rare that users
specify this parameter to less than 1 second. I guess it would be a
good idea to accept string representation of a duration too such as
'10 min' like we do for parsing GUC parameter values.
---
+static void
+notify_ineffective_max_conflict_retention(bool update_maxconflretention)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ update_maxconflretention
+ ? errmsg("max_conflict_retention_duration has no effect
when retain_dead_tuples is disabled")
+ : errmsg("disabling retain_dead_tuples will render
max_conflict_retention_duration ineffective"));
+}
Given that max_conflict_retention_duration works only when
retain_dead_tuples is enabled, why not merge these two parameters? For
example, setting max_conflict_retention_duration=-1 means to disable
retaining dead tuples behavior and =0 means that dead tuples are
retained until they are no longer needed for detection purposes.
---
+ /*
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
I think that adding a new column to the pg_stat_subscription view
should be implemented in a separate patch since it needs to bump the
catalog version while introducing max_conflict_retention_duration
subscription option doesn't require that.
---
Even if an apply worker disables retaining dead tuples due to
max_conflict_retention_duration, it enables again after the server
restarts. I guess restarting a server doesn't necessarily mean that
the subscription caught up to the publisher and can resume retaining
dead tuples but is this behavior intentional?
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Saturday, August 16, 2025 7:44 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Here are review comments on v62 patch:
Thanks for the comments!
--- + else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) && + strcmp(defel->defname, "max_conflict_retention_duration") == 0) + { + if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION)) + errorConflictingDefElem(defel, pstate); + + opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION; + opts->maxconflretention = defGetInt32(defel); + }The new subscription parameter accepts only integers and takes it as
milliseconds, but I think it would be relatively rare that users
specify this parameter to less than 1 second. I guess it would be a
good idea to accept string representation of a duration too such as
'10 min' like we do for parsing GUC parameter values.
We can consider implementing this. However, currently, other similar non-GUC
time-based options do not support unit specification, such as
autovacuum_vacuum_cost_delay and log_autovacuum_min_duration. As such, including
it in max_conf_xx_retention would require new parsing logic. Perhaps we can
treat this as a separate improvement and explore its implementation later, based
on user feedback ?
Best Regards,
Hou zj
On Sat, Aug 16, 2025 at 5:15 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Regarding the subscription-level option vs. GUC, I don't disagree with
the current approach.For the record, while I agree that the subscription-level option is
more consistent with the existing retain_dead_tuples option and can
work for different requirements, my biggest concern is that if users
set different values to different subscriptions, they might think it
doesn't work as expected. This is because even if a subscription
disables retaining dead tuples due to max_conflict_retention_duration,
the database cluster doesn't stop retaining dead tuples unless all
other subscriptions disable it, meaning that the performance on that
database might not get recovered. I proposed the GUC parameter as I
thought it's less confusing from a user perspective. I'm not sure it's
sufficient to mention that in the documentation but I don't have a
better idea.
I think we might want to gave a GUC as well in the future as both
(subscription option and GUC) can be used in different scenarios but
better to wait for some user feedback before going that far. We can
document this in the option to make users aware how to use it in such
situations.
--- +static void +notify_ineffective_max_conflict_retention(bool update_maxconflretention) +{ + ereport(NOTICE, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + update_maxconflretention + ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled") + : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective")); +}Given that max_conflict_retention_duration works only when
retain_dead_tuples is enabled, why not merge these two parameters? For
example, setting max_conflict_retention_duration=-1 means to disable
retaining dead tuples behavior and =0 means that dead tuples are
retained until they are no longer needed for detection purposes.
I think it can be a source of confusion for users, if not now, then in
the future. Consider that in the future, we also have a GUC to set the
retention duration which will be used for all subscriptions. Now, say,
we define the behaviour such that if this value is set for
subscription, then use that, otherwise, use the GUC value. Then, with
this proposal, if the user sets max_conflict_retention_duration=0, it
will lead to retaining tuples until they are no longer needed but with
the behaviour proposed in patch, one could have simply set
retain_dead_tuples=true and used the GUC value. I understand that it
is debatable how we will design the GUC behaviour in future but I used
it as an example how trying to achieve different things with one
option can be a source of confusion. Even if we decide not to
introduce GUC or define its behaviour differently, I find having
different options in this case is easy to understand and use.
--- + /* + * Only the leader apply worker manages conflict retention (see + * maybe_advance_nonremovable_xid() for details). + */ + if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker)) + values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid); + else + nulls[10] = true;I think that adding a new column to the pg_stat_subscription view
should be implemented in a separate patch since it needs to bump the
catalog version while introducing max_conflict_retention_duration
subscription option doesn't require that.
Won't following change in pg_subsciption.h anyway requires us to bump
catversion?
@@ -81,6 +81,10 @@
CATALOG(pg_subscription,6100,SubscriptionRelationId)
BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
---
Even if an apply worker disables retaining dead tuples due to
max_conflict_retention_duration, it enables again after the server
restarts.
I also find this behaviour questionable because this also means that
it is possible that before restart one would deduce that the
update_deleted conflict won't be reliably detected for a particular
subscription but after restart it could lead to the opposite
conclusion. But note that to make it behave similarly we need to store
this value persistently in pg_subscription unless you have better
ideas for this. Theoretically, there are two places where we can
persist this information, one is with pg_subscription, and other in
origin. I find it is closer to pg_subscription.
--
With Regards,
Amit Kapila.
On Mon, Aug 18, 2025 at 10:36 AM Amit Kapila <amit.kapila16@gmail.com> wrote:
Given that max_conflict_retention_duration works only when
retain_dead_tuples is enabled, why not merge these two parameters? For
example, setting max_conflict_retention_duration=-1 means to disable
retaining dead tuples behavior and =0 means that dead tuples are
retained until they are no longer needed for detection purposes.I think it can be a source of confusion for users, if not now, then in
the future. Consider that in the future, we also have a GUC to set the
retention duration which will be used for all subscriptions. Now, say,
we define the behaviour such that if this value is set for
subscription, then use that, otherwise, use the GUC value. Then, with
this proposal, if the user sets max_conflict_retention_duration=0, it
will lead to retaining tuples until they are no longer needed but with
the behaviour proposed in patch, one could have simply set
retain_dead_tuples=true and used the GUC value. I understand that it
is debatable how we will design the GUC behaviour in future but I used
it as an example how trying to achieve different things with one
option can be a source of confusion. Even if we decide not to
introduce GUC or define its behaviour differently, I find having
different options in this case is easy to understand and use.
I agree that merging these 2 will create confusion and usability issues.
--- + /* + * Only the leader apply worker manages conflict retention (see + * maybe_advance_nonremovable_xid() for details). + */ + if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker)) + values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid); + else + nulls[10] = true;I think that adding a new column to the pg_stat_subscription view
should be implemented in a separate patch since it needs to bump the
catalog version while introducing max_conflict_retention_duration
subscription option doesn't require that.Won't following change in pg_subsciption.h anyway requires us to bump
catversion?
@@ -81,6 +81,10 @@
CATALOG(pg_subscription,6100,SubscriptionRelationId)
BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */+ int32 maxconflretention; /* The maximum duration (in milliseconds) + * for which information useful for + * conflict detection can be retained */ +---
Even if an apply worker disables retaining dead tuples due to
max_conflict_retention_duration, it enables again after the server
restarts.I also find this behaviour questionable because this also means that
it is possible that before restart one would deduce that the
update_deleted conflict won't be reliably detected for a particular
subscription but after restart it could lead to the opposite
conclusion. But note that to make it behave similarly we need to store
this value persistently in pg_subscription unless you have better
ideas for this. Theoretically, there are two places where we can
persist this information, one is with pg_subscription, and other in
origin. I find it is closer to pg_subscription.
I think it makes sense to store this in pg_subscription to preserve
the decision across restart.
--
Regards,
Dilip Kumar
Google
On Monday, August 18, 2025 2:32 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Aug 18, 2025 at 10:36 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:---
Even if an apply worker disables retaining dead tuples due to
max_conflict_retention_duration, it enables again after the server
restarts.I also find this behaviour questionable because this also means that
it is possible that before restart one would deduce that the
update_deleted conflict won't be reliably detected for a particular
subscription but after restart it could lead to the opposite
conclusion. But note that to make it behave similarly we need to store
this value persistently in pg_subscription unless you have better
ideas for this. Theoretically, there are two places where we can
persist this information, one is with pg_subscription, and other in
origin. I find it is closer to pg_subscription.I think it makes sense to store this in pg_subscription to preserve the decision
across restart.
Thanks for sharing the opinion!
Regarding this, I'd like to clarify some implementation details for persisting the
retention status in pg_subscription.
Since the logical launcher does not connect to a specific database, it cannot
update the catalog, as this would trigger a FATAL error (e.g.,
CatalogTupleUpdate -> ... -> ScanPgRelation -> FATAL: cannot read pg_class
without having selected a database). Therefore, the apply worker should take
responsibility for updating the catalog.
To achieve that, ideally, the apply worker should update pg_subscription in a
separate transaction, rather than using the transaction started during the
application of changes. This implies that we must wait for the current
transaction to complete before proceeding with the catalog update. So I think we
could an additional phase, RDT_MARK_RETENTION_INACTIVE, to manage the
catalog update once the existing transaction finishes.
If we proceed in this manner, it suggests that the apply worker could set the
shared memory flag first and then catalog flag. So, if the apply worker
encounters an error after setting the shared memory flag but before updating the
catalog, it may lead to issues similar to the one mentioned by Sawada-San,
e.g., the apply worker restart but would retain the dead tuples again because
the status had not persisted. This seems like a rare case, so I'm not sure
whether it's necessary to address it. If we do decide to handle it, we could
update the catalog upon an ERROR using a PG_CATCH block, similar to
DisableSubscriptionAndExit().
Another way could be remove the shared flag and simply depend on the catalog
flag. The launcher will only check the retention in the catalog to decide
whether to invalidate the slot or skip collecting oldest_xid for a apply worker.
However, this approach could retain the dead tuples for more time than the
specified value in max_retention option, due to waiting for a large transaction
to finish before updating the catalog.
What do you think ?
Best Regards,
Hou zj
On Mon, Aug 18, 2025 at 5:05 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, August 18, 2025 2:32 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Mon, Aug 18, 2025 at 10:36 AM Amit Kapila <amit.kapila16@gmail.com>
wrote:---
Even if an apply worker disables retaining dead tuples due to
max_conflict_retention_duration, it enables again after the server
restarts.I also find this behaviour questionable because this also means that
it is possible that before restart one would deduce that the
update_deleted conflict won't be reliably detected for a particular
subscription but after restart it could lead to the opposite
conclusion. But note that to make it behave similarly we need to store
this value persistently in pg_subscription unless you have better
ideas for this. Theoretically, there are two places where we can
persist this information, one is with pg_subscription, and other in
origin. I find it is closer to pg_subscription.I think it makes sense to store this in pg_subscription to preserve the decision
across restart.Thanks for sharing the opinion!
Regarding this, I'd like to clarify some implementation details for persisting the
retention status in pg_subscription.Since the logical launcher does not connect to a specific database, it cannot
update the catalog, as this would trigger a FATAL error (e.g.,
CatalogTupleUpdate -> ... -> ScanPgRelation -> FATAL: cannot read pg_class
without having selected a database). Therefore, the apply worker should take
responsibility for updating the catalog.To achieve that, ideally, the apply worker should update pg_subscription in a
separate transaction, rather than using the transaction started during the
application of changes. This implies that we must wait for the current
transaction to complete before proceeding with the catalog update. So I think we
could an additional phase, RDT_MARK_RETENTION_INACTIVE, to manage the
catalog update once the existing transaction finishes.If we proceed in this manner, it suggests that the apply worker could set the
shared memory flag first and then catalog flag. So, if the apply worker
encounters an error after setting the shared memory flag but before updating the
catalog, it may lead to issues similar to the one mentioned by Sawada-San,
e.g., the apply worker restart but would retain the dead tuples again because
the status had not persisted.
In this approach, why do we need to set the shared memory flag in the
first place, can't we rely on the catalog values? I understand there
is some delay when we detect to stop retention and when we actually
update the catalog but it shouldn't be big enough to matter for small
transactions because we will update it at the next transaction
boundary. For large transactions, we can always update it at the next
stream_stop message.
--
With Regards,
Amit Kapila.
On Sun, Aug 17, 2025 at 10:06 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Sat, Aug 16, 2025 at 5:15 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
Regarding the subscription-level option vs. GUC, I don't disagree with
the current approach.For the record, while I agree that the subscription-level option is
more consistent with the existing retain_dead_tuples option and can
work for different requirements, my biggest concern is that if users
set different values to different subscriptions, they might think it
doesn't work as expected. This is because even if a subscription
disables retaining dead tuples due to max_conflict_retention_duration,
the database cluster doesn't stop retaining dead tuples unless all
other subscriptions disable it, meaning that the performance on that
database might not get recovered. I proposed the GUC parameter as I
thought it's less confusing from a user perspective. I'm not sure it's
sufficient to mention that in the documentation but I don't have a
better idea.I think we might want to gave a GUC as well in the future as both
(subscription option and GUC) can be used in different scenarios but
better to wait for some user feedback before going that far. We can
document this in the option to make users aware how to use it in such
situations.
Okay.
--- +static void +notify_ineffective_max_conflict_retention(bool update_maxconflretention) +{ + ereport(NOTICE, + errcode(ERRCODE_INVALID_PARAMETER_VALUE), + update_maxconflretention + ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled") + : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective")); +}Given that max_conflict_retention_duration works only when
retain_dead_tuples is enabled, why not merge these two parameters? For
example, setting max_conflict_retention_duration=-1 means to disable
retaining dead tuples behavior and =0 means that dead tuples are
retained until they are no longer needed for detection purposes.I think it can be a source of confusion for users, if not now, then in
the future. Consider that in the future, we also have a GUC to set the
retention duration which will be used for all subscriptions. Now, say,
we define the behaviour such that if this value is set for
subscription, then use that, otherwise, use the GUC value. Then, with
this proposal, if the user sets max_conflict_retention_duration=0, it
will lead to retaining tuples until they are no longer needed but with
the behaviour proposed in patch, one could have simply set
retain_dead_tuples=true and used the GUC value. I understand that it
is debatable how we will design the GUC behaviour in future but I used
it as an example how trying to achieve different things with one
option can be a source of confusion. Even if we decide not to
introduce GUC or define its behaviour differently, I find having
different options in this case is easy to understand and use.
Agreed.
--- + /* + * Only the leader apply worker manages conflict retention (see + * maybe_advance_nonremovable_xid() for details). + */ + if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker)) + values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid); + else + nulls[10] = true;I think that adding a new column to the pg_stat_subscription view
should be implemented in a separate patch since it needs to bump the
catalog version while introducing max_conflict_retention_duration
subscription option doesn't require that.Won't following change in pg_subsciption.h anyway requires us to bump
catversion?
Opps, you're right. I missed that part.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Monday, August 18, 2025 8:41 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 18, 2025 at 5:05 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Monday, August 18, 2025 2:32 PM Dilip Kumar <dilipbalaut@gmail.com>
wrote:
On Mon, Aug 18, 2025 at 10:36 AM Amit Kapila
<amit.kapila16@gmail.com>
wrote:
---
Even if an apply worker disables retaining dead tuples due to
max_conflict_retention_duration, it enables again after the server
restarts.I also find this behaviour questionable because this also means that
it is possible that before restart one would deduce that the
update_deleted conflict won't be reliably detected for a particular
subscription but after restart it could lead to the opposite
conclusion. But note that to make it behave similarly we need to store
this value persistently in pg_subscription unless you have better
ideas for this. Theoretically, there are two places where we can
persist this information, one is with pg_subscription, and other in
origin. I find it is closer to pg_subscription.I think it makes sense to store this in pg_subscription to preserve the
decision
across restart.
Thanks for sharing the opinion!
Regarding this, I'd like to clarify some implementation details for persisting
the
retention status in pg_subscription.
Since the logical launcher does not connect to a specific database, it cannot
update the catalog, as this would trigger a FATAL error (e.g.,
CatalogTupleUpdate -> ... -> ScanPgRelation -> FATAL: cannot readpg_class
without having selected a database). Therefore, the apply worker should take
responsibility for updating the catalog.To achieve that, ideally, the apply worker should update pg_subscription in a
separate transaction, rather than using the transaction started during the
application of changes. This implies that we must wait for the current
transaction to complete before proceeding with the catalog update. So I thinkwe
could an additional phase, RDT_MARK_RETENTION_INACTIVE, to manage
the
catalog update once the existing transaction finishes.
If we proceed in this manner, it suggests that the apply worker could set the
shared memory flag first and then catalog flag. So, if the apply worker
encounters an error after setting the shared memory flag but before updatingthe
catalog, it may lead to issues similar to the one mentioned by Sawada-San,
e.g., the apply worker restart but would retain the dead tuples again because
the status had not persisted.In this approach, why do we need to set the shared memory flag in the
first place, can't we rely on the catalog values? I understand there
is some delay when we detect to stop retention and when we actually
update the catalog but it shouldn't be big enough to matter for small
transactions because we will update it at the next transaction
boundary. For large transactions, we can always update it at the next
stream_stop message.
I agree. Here is V63 version which implements this approach.
The retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts. The apply worker is responsible for updating this flag based on the
retention duration. Meanwhile, the column is set to true when retain_dead_tuples
is enabled or when creating a new subscription with retain_dead_tuples enabled,
and it is set to false when retain_dead_tuples is disabled.
Best Regards,
Hou zj
Attachments:
v63-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v63-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From c46ddfc5c135b1994e33a4953ec5c754e04ecfdd Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 20 Aug 2025 14:04:34 +0800
Subject: [PATCH v63 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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/ref/create_subscription.sgml | 4 +
src/backend/replication/logical/launcher.c | 81 +++++++--
src/backend/replication/logical/worker.c | 201 +++++++++++++++++++--
3 files changed, 256 insertions(+), 30 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 52822b2fbb5..f032dc2df59 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -541,6 +541,10 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>max_conflict_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new apply worker with retain_dead_tuples
+ enabled is started.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 20e910e5156..25401a1d3c8 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -1251,6 +1254,14 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ /*
+ * Initialize slot.xmin as a subscription resumes retention of
+ * information useful for conflict detection.
+ */
+ if (sub->retentionactive &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
}
if (!sub->enabled)
@@ -1267,9 +1278,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && sub->retentionactive &&
- can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1380,7 +1390,8 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
@@ -1396,7 +1407,27 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->mutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->mutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->mutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1457,23 +1488,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1493,6 +1516,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 1b2d6a3e001..0ef3806961e 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -177,11 +177,21 @@
* Update pg_subscription.subretentionactive to false within a new
* transaction, and set oldest_nonremovable_xid to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * Update pg_subscription.subretentionactive to true within a new transaction,
+ * and wait for the launcher to initialize the oldest_nonremovable_xid.
+ *
* 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 -> (RDT_STOP_CONFLICT_INFO_RETENTION if the wait time
- * exceeds max_conflict_retention_duration) -> loop back to GET_CANDIDATE_XID.
+ * exceeds max_conflict_retention_duration) ->
+ * (RDT_RESUME_CONFLICT_INFO_RETENTION if the wait time has decreased to within
+ * acceptable limits) -> loop back to GET_CANDIDATE_XID.
+ *
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION at
+ * any phase if the retention has been stopped, but
+ * max_conflict_retention_duration is now set to 0.
*
* Retaining the dead tuples for this period is sufficient for ensuring
* eventual consistency using last-update-wins strategy, as dead tuples are
@@ -380,6 +390,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -424,6 +435,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -567,6 +582,9 @@ static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4361,6 +4379,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4383,10 +4408,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4415,6 +4436,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4671,6 +4695,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4737,6 +4773,112 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
ApplyLauncherWakeup();
reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /*
+ * Proceed to the next phase if the catalog has been updated and the
+ * launcher has initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race
+ * condition where slot.xmin is set to InvalidTransactionId
+ * immediately after the check. In such cases, oldest_nonremovable_xid
+ * would no longer be protected by a replication slot and could become
+ * unreliable if a wraparound occurs.
+ */
+ if (TransactionIdIsValid(nonremovable_xid))
+ {
+ rdt_data->wait_for_initial_xid = false;
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ /*
+ * Return early since the catalog has been updated and we are waiting
+ * for oldest_nonremovable_xid to be initialized.
+ */
+ return;
+ }
+
+ /*
+ * Proceed to the next phase if retention has not been stopped yet. This
+ * occurs when transitioning from the RDT_STOP_CONFLICT_INFO_RETENTION
+ * phase but subretentionactive has not been updated due to the inability
+ * to start a new transaction (see stop_conflict_info_retention).
+ */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return;
+ }
+
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to true */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, true);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxconflretention
+ ? errdetail("The retention duration for information used in conflict detection is now within the acceptable limit of %u ms.",
+ MySubscription->maxconflretention)
+ : errdetail("The retention duration for information used in conflict detection is now indefinite."));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
+
+ /*
+ * The next step is to wait for the launcher to initialize the
+ * oldest_nonremovable_xid.
+ */
+ rdt_data->wait_for_initial_xid = true;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
}
/*
@@ -4766,9 +4908,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4799,14 +4940,52 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
return true;
}
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_conflict_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_conflict_retention_duration is not 0 */
+ if (MySubscription->maxconflretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.50.1.windows.1
v63-0001-Introduce-a-max_conflict_retention_duration-opti.patchapplication/octet-stream; name=v63-0001-Introduce-a-max_conflict_retention_duration-opti.patchDownload
From ad2c50f99fe504a57e9480a7c52a307ecca78f4c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v63] Introduce a 'max_conflict_retention_duration' option to
subscriptions.
This commit introduces a subscription option max_conflict_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization 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 'dead_tuple_retention_active' 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_dead_tuples 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/catalogs.sgml | 25 +++
doc/src/sgml/monitoring.sgml | 12 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 42 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 88 +++++++-
src/backend/replication/logical/launcher.c | 53 +++--
src/backend/replication/logical/tablesync.c | 22 +-
src/backend/replication/logical/worker.c | 225 ++++++++++++++++++--
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 19 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/replication/worker_internal.h | 10 +-
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 186 +++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
src/test/subscription/t/035_conflicts.pl | 10 +-
22 files changed, 672 insertions(+), 139 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..98bfa96b17b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..5f93b918274 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..f2c2e147472 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..52822b2fbb5 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,46 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..16eb5c16a0b 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxconflretention = subform->submaxconflretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..ccc78d81839 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
FROM pg_subscription su
LEFT JOIN pg_stat_get_subscription(NULL) st
ON (st.subid = su.oid);
@@ -1389,8 +1390,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxconflretention, subretentionactive,
+ 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 faa3650d287..5b97f1d4ba2 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxconflretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -112,6 +114,7 @@ static void check_publications_origin(WalReceiverConn *wrconn,
Oid *subrel_local_oids, int subrel_count,
char *subname);
static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
+static void notify_ineffective_max_conflict_retention(bool update_maxconflretention);
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);
@@ -169,6 +172,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +328,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +594,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -651,6 +666,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
if (opts.retaindeadtuples)
CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /* Notify that max_conflict_retention_duration is ineffective */
+ else if (opts.maxconflretention)
+ notify_ineffective_max_conflict_retention(true);
+
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
opts.slot_name = stmt->subname;
@@ -693,6 +712,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_failover = false;
bool update_two_phase = false;
bool check_pub_rdt = false;
+ bool ineffective_maxconflretention = false;
+ bool update_maxconflretention = false;
bool retain_dead_tuples;
char *origin;
Subscription *sub;
@@ -1235,7 +1260,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1401,6 +1428,26 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only when there is a change
+ * in the retain_dead_tuples option value.
+ *
+ * It might not be ideal to blindly mark retention as active
+ * upon enabling the retain_dead_tuples, when retention was
+ * previously ceased and the user toggles retain_dead_tuples
+ * without adjusting the publisher workload. However, since
+ * retention will be stopped gain soon in such cases, and
+ * this approach offers a convenient way for the user to
+ * manually refresh the retention status, it is suitable for
+ * now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1434,6 +1481,20 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
check_pub_rdt = opts.retaindeadtuples;
retain_dead_tuples = opts.retaindeadtuples;
+
+ ineffective_maxconflretention = (!opts.retaindeadtuples &&
+ sub->maxconflretention);
+ }
+
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ replaces[Anum_pg_subscription_submaxconflretention - 1] = true;
+
+ update_maxconflretention = true;
+ ineffective_maxconflretention = (!retain_dead_tuples &&
+ opts.maxconflretention);
}
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
@@ -1453,6 +1514,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
origin = opts.origin;
}
+ if (ineffective_maxconflretention)
+ notify_ineffective_max_conflict_retention(update_maxconflretention);
+
update_tuple = true;
break;
}
@@ -2500,6 +2564,22 @@ CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
"retain_dead_tuples") : 0);
}
+/*
+ * Report a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_conflict_retention_duration causes no harm,
+ * even when it is ineffective.
+ */
+static void
+notify_ineffective_max_conflict_retention(bool update_maxconflretention)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ update_maxconflretention
+ ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled")
+ : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective"));
+}
+
/*
* 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 37377f7eb63..20e910e5156 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -998,7 +1001,7 @@ ApplyLauncherShmemInit(void)
LogicalRepWorker *worker = &LogicalRepCtx->workers[slot];
memset(worker, 0, sizeof(LogicalRepWorker));
- SpinLockInit(&worker->relmutex);
+ SpinLockInit(&worker->mutex);
}
}
}
@@ -1183,6 +1186,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool retention_inactive = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1225,6 +1229,13 @@ ApplyLauncherMain(Datum main_arg)
*/
can_advance_xmin &= sub->enabled;
+ /*
+ * Consider overall retention inactive only when all
+ * subscriptions with retain_dead_tuples enabled have marked
+ * it as inactive.
+ */
+ retention_inactive &= !sub->retentionactive;
+
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1256,7 +1267,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples && sub->retentionactive &&
+ can_advance_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1320,13 +1332,19 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention,
+ * marking it as inactive, the new xmin will be set to
+ * InvalidTransactionId. We then update slot.xmin accordingly to
+ * permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_advance_xmin || retention_inactive)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1374,9 +1392,9 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
*/
Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
+ SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockRelease(&worker->mutex);
Assert(TransactionIdIsValid(nonremovable_xid));
@@ -1402,17 +1420,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1536,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;
@@ -1595,6 +1613,15 @@ 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))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3356bc84ee..1ab5496f63f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -293,7 +293,7 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
@@ -305,7 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
MyLogicalRepWorker->relstate_lsn = current_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* UpdateSubscriptionRelState must be called within a transaction.
@@ -390,7 +390,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
finish_sync_worker();
}
else
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
}
/*
@@ -534,7 +534,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
if (syncworker)
{
/* Found one, update our copy of its state */
- SpinLockAcquire(&syncworker->relmutex);
+ SpinLockAcquire(&syncworker->mutex);
rstate->state = syncworker->relstate;
rstate->lsn = syncworker->relstate_lsn;
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -547,7 +547,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
syncworker->relstate_lsn =
Max(syncworker->relstate_lsn, current_lsn);
}
- SpinLockRelease(&syncworker->relmutex);
+ SpinLockRelease(&syncworker->mutex);
/* If we told worker to catch up, wait for it. */
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -1342,10 +1342,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
must_use_password = MySubscription->passwordrequired &&
!MySubscription->ownersuperuser;
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = relstate;
MyLogicalRepWorker->relstate_lsn = relstate_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* If synchronization is already done or no longer necessary, exit now
@@ -1428,10 +1428,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
goto copy_table_done;
}
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/* Update the state and make it visible to others. */
StartTransactionCommand();
@@ -1586,10 +1586,10 @@ copy_table_done:
/*
* We are done with the initial data synchronization, update the state.
*/
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCWAIT;
MyLogicalRepWorker->relstate_lsn = *origin_startpos;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* Finally, wait until the leader apply worker tells us to catch up and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 8e343873454..1b2d6a3e001 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,10 +173,15 @@
* Advance the non-removable transaction ID if the current flush location has
* reached or surpassed the last received WAL position.
*
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * Update pg_subscription.subretentionactive to false within a new
+ * transaction, and set oldest_nonremovable_xid to InvalidTransactionId.
+ *
* 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.
+ * WAIT_FOR_LOCAL_FLUSH -> (RDT_STOP_CONFLICT_INFO_RETENTION if the wait time
+ * exceeds max_conflict_retention_duration) -> 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
@@ -373,7 +378,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +421,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +564,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3232,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool stop_retention;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3241,42 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ */
+ if (am_leader_apply_worker())
+ {
+ stop_retention =
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->mutex);
+ stop_retention = !TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->mutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (stop_retention)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3303,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4167,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4383,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4412,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4533,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4621,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4583,9 +4676,9 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%08X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rdt_data->remote_lsn),
@@ -4594,12 +4687,67 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
*/
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4755,56 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
+
+/*
+ * 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, proceed to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxconflretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxconflretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
+
+ return true;
}
/*
@@ -4621,8 +4816,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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.
+ * The interval is reset to the lesser of 100ms and
+ * max_conflict_retention_duration once there is some activities 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.
@@ -4642,6 +4837,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
}
else
{
@@ -5455,7 +5654,7 @@ InitializeLogRepWorker(void)
* logicalrep_worker_launch.
*/
if (am_leader_apply_worker() &&
- MySubscription->retaindeadtuples &&
+ MySubscription->retaindeadtuples && MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..8d8bcf61075 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxconflretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxconflretention = PQfnumber(res, "submaxconflretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxconflretention =
+ atoi(PQgetvalue(res, i, i_submaxconflretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxconflretention)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflretention);
+
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 dde85ed156c..e6b94422af7 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxconflretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..b6d57d02778 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflretention AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Dead tuple retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..2d7016fe717 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..fcc1ad173ca 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,17 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxconflretention; /* The maximum duration (in
+ * milliseconds) for which information
+ * useful for conflict detection can
+ * be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded
+ * max_conflict_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +147,14 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded
+ * max_conflict_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..9c0c2b8050c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -64,7 +64,12 @@ typedef struct LogicalRepWorker
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
- slock_t relmutex;
+
+ /*
+ * Spinlock used to protect table synchronization information and the
+ * oldest_nonremovable_xid.
+ */
+ slock_t mutex;
/*
* Used to create the changes and subxact files for the streaming
@@ -94,6 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c6b0784b253 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+NOTICE: max_conflict_retention_duration has no effect when retain_dead_tuples 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 dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..9b2c489adaf 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.50.1.windows.1
On Wed, Aug 20, 2025 at 12:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I agree. Here is V63 version which implements this approach.
Thank You for the patches.
The retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts. The apply worker is responsible for updating this flag based on the
retention duration. Meanwhile, the column is set to true when retain_dead_tuples
is enabled or when creating a new subscription with retain_dead_tuples enabled,
and it is set to false when retain_dead_tuples is disabled.
+1 on the idea.
Please find few initial testing feedback:
1)
When it stops, it does not resume until we restart th server. It keeps
on waiting in wait_for_publisher_status and it never receives one.
2)
When we do: alter subscription sub1 set (max_conflict_retention_duration=0);
It does not resume in this scenario too.
should_resume_retention_immediately() does not return true due to
wait-status on publisher.
3)
AlterSubscription():
* retention will be stopped gain soon in such cases, and
stopped gain --> stopped again
thanks
Shveta
On Wed, Aug 20, 2025 at 12:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I agree. Here is V63 version which implements this approach.
Thank you Hou-san for the patches. Here are couple of comments:
1) Once retention is stopped for all subscriptions and
conflict_slot.xmin is reset to NULL, we are no longer retaining dead
tuples. In that case, the warning shown during subscription disable
looks misleading.
For example sub has already stopped the retention and when disabled -
postgres=# alter subscription sub1 disable;
WARNING: deleted rows to detect conflicts would not be removed until
the subscription is enabled
HINT: Consider setting retain_dead_tuples to false.
ALTER SUBSCRIPTION
I think we should check if retention is active or not here.
2) Regarding the logic in the launcher for advancing the slot’s xmin:
Consider a case where two subscriptions exist, and one of them is
disabled after it has already stopped retention.
Example subscriptions in state:
subname | subenabled | subretaindeadtuples | submaxconflretention |
subretentionactive
---------+------------+---------------------+----------------------+--------------------
sub1 | t | t | 100 | t
sub2 | f | t | 100 | f
Here, sub2 is disabled, and since subretentionactive = 'f', it is not
retaining dead tuples anymore. But, the current launcher logic still
blocks xmin advancement as one of the subscriptions with
retain_dead_tuples is disabled.
I think the launcher should consider the subretentionactive value and
the xmin should be allowed to advance. Thoughts?
--
Thanks,
Nisha
On Thursday, August 21, 2025 3:47 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
On Wed, Aug 20, 2025 at 12:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I agree. Here is V63 version which implements this approach.
Thank you Hou-san for the patches. Here are couple of comments:
1) Once retention is stopped for all subscriptions and conflict_slot.xmin is
reset to NULL, we are no longer retaining dead tuples. In that case, the warning
shown during subscription disable looks misleading.For example sub has already stopped the retention and when disabled -
postgres=# alter subscription sub1 disable;
WARNING: deleted rows to detect conflicts would not be removed until the
subscription is enabled
HINT: Consider setting retain_dead_tuples to false.
ALTER SUBSCRIPTIONI think we should check if retention is active or not here.
2) Regarding the logic in the launcher for advancing the slot’s xmin:
Consider a case where two subscriptions exist, and one of them is disabled
after it has already stopped retention.
Example subscriptions in state:
...
Here, sub2 is disabled, and since subretentionactive = 'f', it is not retaining
dead tuples anymore. But, the current launcher logic still blocks xmin
advancement as one of the subscriptions with retain_dead_tuples is disabled.
I think the launcher should consider the subretentionactive value and the xmin
should be allowed to advance. Thoughts?
I agree that retentionactive needs to be checked in the cases mentioned above.
Here is the V64 patch set addressing this concern. This version also resolves
the bug reported by Shveta[1]/messages/by-id/CAJpy0uCP7x_pdVysYohvrjpk0Vtmk36+XfnC_DOPiegekxfBLA@mail.gmail.com, where retention could not resume and was stuck
waiting for the publisher status.
In addition, I also improved the comments related to the new phases and
retentionactive flag.
[1]: /messages/by-id/CAJpy0uCP7x_pdVysYohvrjpk0Vtmk36+XfnC_DOPiegekxfBLA@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v64-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v64-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 605c2759160fce0fcdb8c38babf0da8f8bed10c5 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Thu, 21 Aug 2025 15:23:02 +0800
Subject: [PATCH v64 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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/ref/create_subscription.sgml | 7 +-
src/backend/replication/logical/launcher.c | 81 ++++++--
src/backend/replication/logical/worker.c | 204 +++++++++++++++++++--
3 files changed, 260 insertions(+), 32 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 8de0cd0d53f..f032dc2df59 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -540,8 +540,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
retention duration exceeded the
<literal>max_conflict_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
- disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new subscription with retain_dead_tuples
+ enabled is created.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 81cae445cc0..3a86300a4fa 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -1252,6 +1255,14 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ /*
+ * Initialize slot.xmin as a subscription resumes retention of
+ * information useful for conflict detection.
+ */
+ if (sub->retentionactive &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
}
if (!sub->enabled)
@@ -1268,9 +1279,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && sub->retentionactive &&
- can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1381,7 +1391,8 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
@@ -1397,7 +1408,27 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->mutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->mutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->mutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1458,23 +1489,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1494,6 +1517,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c7c9a0be356..0c2dcf4a0e3 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -194,7 +194,7 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
- * If max_conflict_retention_duration is defined, one additional phase is
+ * If max_conflict_retention_duration is defined, two additional phases are
* involved:
*
* - RDT_STOP_CONFLICT_INFO_RETENTION:
@@ -204,6 +204,20 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_conflict_retention_duration is set to 0). Additionally, it
+ * initiates if conditions improve allowing more efficient advancement. During
+ * this phase, pg_subscription.subretentionactive is updated to true within a
+ * new transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ *
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION at
+ * any phase if the retention has been stopped, but
+ * max_conflict_retention_duration is now set to 0.
+ *
* 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
@@ -389,6 +403,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -433,6 +448,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -576,6 +595,9 @@ static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4377,6 +4399,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4399,10 +4428,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4431,6 +4456,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4687,6 +4715,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4753,6 +4793,109 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
ApplyLauncherWakeup();
reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ /*
+ * Proceed to the next phase if the catalog has been updated and the
+ * launcher has initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race
+ * condition where slot.xmin is set to InvalidTransactionId
+ * immediately after the check. In such cases, oldest_nonremovable_xid
+ * would no longer be protected by a replication slot and could become
+ * unreliable if a wraparound occurs.
+ */
+ if (TransactionIdIsValid(nonremovable_xid))
+ {
+ rdt_data->wait_for_initial_xid = false;
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ /*
+ * Return early since the catalog has been updated and we are waiting
+ * for oldest_nonremovable_xid to be initialized.
+ */
+ return;
+ }
+
+ /*
+ * Proceed to the next phase if retention has not been stopped yet. This
+ * occurs when transitioning from the RDT_STOP_CONFLICT_INFO_RETENTION
+ * phase but subretentionactive has not been updated due to the inability
+ * to start a new transaction (see stop_conflict_info_retention).
+ */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return;
+ }
+
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to true */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, true);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxconflretention
+ ? errdetail("The retention duration for information used in conflict detection is now within the acceptable limit of %u ms.",
+ MySubscription->maxconflretention)
+ : errdetail("The retention duration for information used in conflict detection is now indefinite."));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
+
+ /*
+ * The next step is to wait for the launcher to initialize the
+ * oldest_nonremovable_xid.
+ */
+ rdt_data->wait_for_initial_xid = true;
}
/*
@@ -4782,9 +4925,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4815,14 +4957,54 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_conflict_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_conflict_retention_duration is not 0 */
+ if (MySubscription->maxconflretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.31.1
v64-0001-Introduce-a-max_conflict_retention_duration-opti.patchapplication/octet-stream; name=v64-0001-Introduce-a-max_conflict_retention_duration-opti.patchDownload
From 405c37e54859137d69f300ca7325d167bf4093fb Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Wed, 23 Jul 2025 13:38:12 +0800
Subject: [PATCH v64] Introduce a 'max_conflict_retention_duration' option to
subscriptions.
This commit introduces a subscription option max_conflict_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization 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 'dead_tuple_retention_active' 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_dead_tuples 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/catalogs.sgml | 25 ++
doc/src/sgml/monitoring.sgml | 12 +
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 7 +-
src/backend/commands/subscriptioncmds.c | 111 +++++++--
src/backend/replication/logical/launcher.c | 73 ++++--
src/backend/replication/logical/tablesync.c | 22 +-
src/backend/replication/logical/worker.c | 242 ++++++++++++++++++--
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_proc.dat | 6 +-
src/include/catalog/pg_subscription.h | 19 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 3 +-
src/include/replication/worker_internal.h | 10 +-
src/test/regress/expected/rules.out | 5 +-
src/test/regress/expected/subscription.out | 186 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
src/test/subscription/t/035_conflicts.pl | 10 +-
23 files changed, 720 insertions(+), 155 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..98bfa96b17b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..5f93b918274 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..f2c2e147472 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..8de0cd0d53f 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..16eb5c16a0b 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxconflretention = subform->submaxconflretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..ccc78d81839 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
FROM pg_subscription su
LEFT JOIN pg_stat_get_subscription(NULL) st
ON (st.subid = su.oid);
@@ -1389,8 +1390,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxconflretention, subretentionactive,
+ 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 4c01d21b2f3..21434b5df93 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxconflretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -112,6 +114,7 @@ static void check_publications_origin(WalReceiverConn *wrconn,
Oid *subrel_local_oids, int subrel_count,
char *subname);
static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
+static void notify_ineffective_max_conflict_retention(bool update_maxconflretention);
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);
@@ -169,6 +172,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +328,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +594,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -649,7 +664,11 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
/* Ensure that we can enable retain_dead_tuples */
if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING, true);
+
+ /* Notify that max_conflict_retention_duration is ineffective */
+ else if (opts.maxconflretention)
+ notify_ineffective_max_conflict_retention(true);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -693,6 +712,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_failover = false;
bool update_two_phase = false;
bool check_pub_rdt = false;
+ bool ineffective_maxconflretention = false;
+ bool update_maxconflretention = false;
bool retain_dead_tuples;
char *origin;
Subscription *sub;
@@ -1235,7 +1260,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1397,10 +1424,34 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
{
+ bool retention_active = sub->retentionactive;
+
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only when there is a change
+ * in the retain_dead_tuples option value.
+ *
+ * It might not be ideal to blindly mark retention as
+ * active upon enabling the retain_dead_tuples, when
+ * retention was previously ceased and the user toggles
+ * retain_dead_tuples without adjusting the publisher
+ * workload. However, since retention will be stopped gain
+ * soon in such cases, and this approach offers a
+ * convenient way for the user to manually refresh the
+ * retention status, it is suitable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1422,7 +1473,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* the accumulation of dead tuples.
*/
if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retention_active);
/*
* Notify the launcher to manage the replication slot for
@@ -1434,6 +1486,20 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
check_pub_rdt = opts.retaindeadtuples;
retain_dead_tuples = opts.retaindeadtuples;
+
+ ineffective_maxconflretention = (!opts.retaindeadtuples &&
+ sub->maxconflretention);
+ }
+
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ replaces[Anum_pg_subscription_submaxconflretention - 1] = true;
+
+ update_maxconflretention = true;
+ ineffective_maxconflretention = (!retain_dead_tuples &&
+ opts.maxconflretention);
}
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
@@ -1453,6 +1519,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
origin = opts.origin;
}
+ if (ineffective_maxconflretention)
+ notify_ineffective_max_conflict_retention(update_maxconflretention);
+
update_tuple = true;
break;
}
@@ -1475,7 +1544,7 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
*/
if (sub->retaindeadtuples)
CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ WARNING, sub->retentionactive);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2468,14 +2537,14 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled, bool retention_active)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
@@ -2493,7 +2562,7 @@ CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
errhint("Consider setting \"%s\" to true.",
"track_commit_timestamp"));
- if (sub_disabled)
+ if (sub_disabled && retention_active)
ereport(elevel_for_sub_disabled,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
@@ -2502,6 +2571,22 @@ CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
"retain_dead_tuples") : 0);
}
+/*
+ * Report a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_conflict_retention_duration causes no harm,
+ * even when it is ineffective.
+ */
+static void
+notify_ineffective_max_conflict_retention(bool update_maxconflretention)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ update_maxconflretention
+ ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled")
+ : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective"));
+}
+
/*
* 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 37377f7eb63..81cae445cc0 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -998,7 +1001,7 @@ ApplyLauncherShmemInit(void)
LogicalRepWorker *worker = &LogicalRepCtx->workers[slot];
memset(worker, 0, sizeof(LogicalRepWorker));
- SpinLockInit(&worker->relmutex);
+ SpinLockInit(&worker->mutex);
}
}
}
@@ -1183,6 +1186,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool retention_inactive = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1216,14 +1220,22 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
+ * actively retaining dead tuples are enabled. This is
+ * required to ensure that we don't advance the xmin of
* CONFLICT_DETECTION_SLOT if one of the subscriptions is not
* enabled. Otherwise, we won't be able to detect conflicts
* reliably for such a subscription even though it has set the
* retain_dead_tuples option.
*/
- can_advance_xmin &= sub->enabled;
+ if (!sub->enabled && sub->retentionactive)
+ can_advance_xmin = false;
+
+ /*
+ * Consider overall retention inactive only when all
+ * subscriptions with retain_dead_tuples enabled have marked
+ * it as inactive.
+ */
+ retention_inactive &= !sub->retentionactive;
/*
* Create a replication slot to retain information necessary
@@ -1256,7 +1268,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples && sub->retentionactive &&
+ can_advance_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,11 +1278,11 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
+ if (sub->retaindeadtuples && sub->retentionactive)
can_advance_xmin = false;
/*
@@ -1320,13 +1333,19 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention,
+ * marking it as inactive, the new xmin will be set to
+ * InvalidTransactionId. We then update slot.xmin accordingly to
+ * permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_advance_xmin || retention_inactive)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1374,9 +1393,9 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
*/
Assert(MyReplicationSlot);
- SpinLockAcquire(&worker->relmutex);
+ SpinLockAcquire(&worker->mutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
- SpinLockRelease(&worker->relmutex);
+ SpinLockRelease(&worker->mutex);
Assert(TransactionIdIsValid(nonremovable_xid));
@@ -1402,17 +1421,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1518,7 +1537,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;
@@ -1595,6 +1614,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/backend/replication/logical/tablesync.c b/src/backend/replication/logical/tablesync.c
index d3356bc84ee..1ab5496f63f 100644
--- a/src/backend/replication/logical/tablesync.c
+++ b/src/backend/replication/logical/tablesync.c
@@ -293,7 +293,7 @@ invalidate_syncing_table_states(Datum arg, int cacheid, uint32 hashvalue)
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
if (MyLogicalRepWorker->relstate == SUBREL_STATE_CATCHUP &&
current_lsn >= MyLogicalRepWorker->relstate_lsn)
@@ -305,7 +305,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCDONE;
MyLogicalRepWorker->relstate_lsn = current_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* UpdateSubscriptionRelState must be called within a transaction.
@@ -390,7 +390,7 @@ process_syncing_tables_for_sync(XLogRecPtr current_lsn)
finish_sync_worker();
}
else
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
}
/*
@@ -534,7 +534,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
if (syncworker)
{
/* Found one, update our copy of its state */
- SpinLockAcquire(&syncworker->relmutex);
+ SpinLockAcquire(&syncworker->mutex);
rstate->state = syncworker->relstate;
rstate->lsn = syncworker->relstate_lsn;
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -547,7 +547,7 @@ process_syncing_tables_for_apply(XLogRecPtr current_lsn)
syncworker->relstate_lsn =
Max(syncworker->relstate_lsn, current_lsn);
}
- SpinLockRelease(&syncworker->relmutex);
+ SpinLockRelease(&syncworker->mutex);
/* If we told worker to catch up, wait for it. */
if (rstate->state == SUBREL_STATE_SYNCWAIT)
@@ -1342,10 +1342,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
must_use_password = MySubscription->passwordrequired &&
!MySubscription->ownersuperuser;
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = relstate;
MyLogicalRepWorker->relstate_lsn = relstate_lsn;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* If synchronization is already done or no longer necessary, exit now
@@ -1428,10 +1428,10 @@ LogicalRepSyncTableStart(XLogRecPtr *origin_startpos)
goto copy_table_done;
}
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_DATASYNC;
MyLogicalRepWorker->relstate_lsn = InvalidXLogRecPtr;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/* Update the state and make it visible to others. */
StartTransactionCommand();
@@ -1586,10 +1586,10 @@ copy_table_done:
/*
* We are done with the initial data synchronization, update the state.
*/
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->relstate = SUBREL_STATE_SYNCWAIT;
MyLogicalRepWorker->relstate_lsn = *origin_startpos;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
/*
* Finally, wait until the leader apply worker tells us to catch up and
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..9e67cf694c0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,6 +173,10 @@
* Advance the non-removable transaction ID if the current flush location has
* reached or surpassed the last received WAL position.
*
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * Update pg_subscription.subretentionactive to false within a new
+ * transaction, and set oldest_nonremovable_xid to InvalidTransactionId.
+ *
* 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) ->
@@ -190,6 +194,16 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
+ * If max_conflict_retention_duration is defined, one additional phase is
+ * involved:
+ *
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * max_conflict_retention_duration. During this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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
@@ -373,7 +387,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +430,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +573,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3241,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool retention_active;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3250,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ *
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention is
+ * active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher assigns
+ * the initial oldest_nonremovable_xid after the apply worker updates the
+ * catalog (see resume_conflict_info_retention).
+ */
+ if (am_leader_apply_worker())
+ {
+ retention_active =
+ TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->mutex);
+ retention_active = TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->mutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!retention_active)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3319,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4183,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4399,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4428,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4549,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4637,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4583,9 +4692,9 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
MyLogicalRepWorker->oldest_nonremovable_xid = rdt_data->candidate_xid;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
elog(DEBUG2, "confirmed flush up to remote lsn %X/%08X: new oldest_nonremovable_xid %u",
LSN_FORMAT_ARGS(rdt_data->remote_lsn),
@@ -4594,12 +4703,67 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
*/
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->mutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4771,56 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
+
+/*
+ * 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, proceed to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxconflretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxconflretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
+
+ return true;
}
/*
@@ -4621,8 +4832,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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.
+ * The interval is reset to the lesser of 100ms and
+ * max_conflict_retention_duration once there is some activities 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.
@@ -4642,6 +4853,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
}
else
{
@@ -5462,7 +5677,7 @@ InitializeLogRepWorker(void)
* logicalrep_worker_launch.
*/
if (am_leader_apply_worker() &&
- MySubscription->retaindeadtuples &&
+ MySubscription->retaindeadtuples && MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5634,7 +5849,8 @@ DisableSubscriptionAndExit(void)
* context.
*/
if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retentionactive);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..8d8bcf61075 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxconflretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxconflretention = PQfnumber(res, "submaxconflretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxconflretention =
+ atoi(PQgetvalue(res, i, i_submaxconflretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxconflretention)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflretention);
+
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 dde85ed156c..e6b94422af7 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxconflretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..b6d57d02778 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflretention AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Dead tuple retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..2d7016fe717 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
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/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..fcc1ad173ca 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,17 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxconflretention; /* The maximum duration (in
+ * milliseconds) for which information
+ * useful for conflict detection can
+ * be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded
+ * max_conflict_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +147,14 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded
+ * max_conflict_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..2bd947062f2 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,7 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retention_active);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..9c0c2b8050c 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -64,7 +64,12 @@ typedef struct LogicalRepWorker
Oid relid;
char relstate;
XLogRecPtr relstate_lsn;
- slock_t relmutex;
+
+ /*
+ * Spinlock used to protect table synchronization information and the
+ * oldest_nonremovable_xid.
+ */
+ slock_t mutex;
/*
* Used to create the changes and subxact files for the streaming
@@ -94,6 +99,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c6b0784b253 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+NOTICE: max_conflict_retention_duration has no effect when retain_dead_tuples 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 dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..9b2c489adaf 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.51.0.windows.1
On Thursday, August 21, 2025 2:01 PM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 20, 2025 at 12:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I agree. Here is V63 version which implements this approach.
Thank You for the patches.
The retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon
server restarts. The apply worker is responsible for updating this
flag based on the retention duration. Meanwhile, the column is set to
true when retain_dead_tuples is enabled or when creating a new
subscription with retain_dead_tuples enabled, and it is set to false whenretain_dead_tuples is disabled.
+1 on the idea.
Please find few initial testing feedback:
Thanks for the comments.
1)
When it stops, it does not resume until we restart th server. It keeps on waiting
in wait_for_publisher_status and it never receives one.2)
When we do: alter subscription sub1 set (max_conflict_retention_duration=0);It does not resume in this scenario too.
should_resume_retention_immediately() does not return true due to
wait-status on publisher.
Fixed in the V64 patches.
3)
AlterSubscription():
* retention will be stopped gain soon in such cases, andstopped gain --> stopped again
Sorry, I missed this typo in V64, I will fix it in the next version.
Best Regards,
Hou zj
Hello!
Sorry for being noisy - just want to remind: conflict detection system
(including new updated_deleted) is giving invalid reports because of
the issue related to SnapshotDirty vs btree. So, it is not possible to
rely on that at the moment.
You may check TAP tests here [0]https://commitfest.postgresql.org/patch/5151/ and some explanation here [1]/messages/by-id/CADzfLwWC49oanFSGPTf=6FJoTw-kAnpPZV8nVqAyR5KL68LrHQ@mail.gmail.com and
here [2]/messages/by-id/CADzfLwWuXh8KO=OZvB71pZnQ8nH0NYXfuGbFU6FBiVZUbmuFGg@mail.gmail.com. Set $simulate_race_condition to disable race condition in
tests.
Invalid conflicts so far (valid/detected):
* delete_origin_differs/delete_missing
* update_origin_differs/update_missing
* update_origin_differs/update_deleted
Best regards,
Mikhail.
[0]: https://commitfest.postgresql.org/patch/5151/
[1]: /messages/by-id/CADzfLwWC49oanFSGPTf=6FJoTw-kAnpPZV8nVqAyR5KL68LrHQ@mail.gmail.com
[2]: /messages/by-id/CADzfLwWuXh8KO=OZvB71pZnQ8nH0NYXfuGbFU6FBiVZUbmuFGg@mail.gmail.com
On Thu, Aug 21, 2025 at 2:09 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Thursday, August 21, 2025 2:01 PM shveta malik <shveta.malik@gmail.com> wrote:
On Wed, Aug 20, 2025 at 12:12 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I agree. Here is V63 version which implements this approach.
Thank You for the patches.
The retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon
server restarts. The apply worker is responsible for updating this
flag based on the retention duration. Meanwhile, the column is set to
true when retain_dead_tuples is enabled or when creating a new
subscription with retain_dead_tuples enabled, and it is set to false whenretain_dead_tuples is disabled.
+1 on the idea.
Please find few initial testing feedback:
Thanks for the comments.
1)
When it stops, it does not resume until we restart th server. It keeps on waiting
in wait_for_publisher_status and it never receives one.2)
When we do: alter subscription sub1 set (max_conflict_retention_duration=0);It does not resume in this scenario too.
should_resume_retention_immediately() does not return true due to
wait-status on publisher.Fixed in the V64 patches.
3)
AlterSubscription():
* retention will be stopped gain soon in such cases, andstopped gain --> stopped again
Sorry, I missed this typo in V64, I will fix it in the next version.
Sure. Thanks.
Please find a few more comments:
1)
There is an issue in retention resumption. The issue is observed for a
multi pub-sub setup where one sub is retaining info while another one
has stopped retention. Now even if I set
max_conflict_retention_duration=0 for the one which has stopped
retention, it does not resume. I have attached steps in the txt file.
2)
In the same testcase, sub1 is not resuming otherwise also i.e. even
though if we do not set max_conflict_retention_duration to 0, it
should resume in a while as there is no other txn on pub which is
stuck in commit-phase. In a single pub-sub setup, it works well. Multi
pub-sub setup has this issue.
3)
ApplyLauncherMain() has some processing under 'if
(sub->retaindeadtuples)', all dependent upon sub->retentionactive.
Will it be better to write it as:
if (sub->retaindeadtuples)
{
retain_dead_tuples = true;
CreateConflictDetectionSlot();
if (sub->retentionactive)
{
retention_inactive = false
can_advance_xmin &= sub->enabled;
if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
init_conflict_slot_xmin();
}
}
All 'sub->retentionactive' based logic under one 'if' would be easier
to understand.
thanks
Shveta
Attachments:
On Thu, Aug 21, 2025 at 2:01 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V64 patch set addressing this concern.
Few minor comments:
1.
static void
process_syncing_tables_for_sync(XLogRecPtr current_lsn)
{
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ SpinLockAcquire(&MyLogicalRepWorker->mutex);
Why is this change part of this patch? Please extract it as a separate
patch unless this change is related to this patch.
2.
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;
@@ -1595,6 +1614,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
The theory given in the comment sounds good to me but I still suggest
it is better to extract it into a separate patch, so that we can
analyse/test it separately. Also, it will reduce the patch size as
well.
3.
/* Ensure that we can enable retain_dead_tuples */
if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING, true);
+
+ /* Notify that max_conflict_retention_duration is ineffective */
+ else if (opts.maxconflretention)
+ notify_ineffective_max_conflict_retention(true);
Can't we combine these checks by passing both parameters to
CheckSubDeadTupleRetention() and let that function handle all
inappropriate value cases? BTW, even for other places, see if you can
reduce the length of the function name
notify_ineffective_max_conflict_retention.
--
With Regards,
Amit Kapila.
On Mon, Aug 4, 2025 at 3:11 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 4, 2025 at 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?Before bikeshedding on the name of this option, I would like us to
once again consider whether we should provide this option at
subscription-level or GUC?
Now that we decided that we would like to go with the subscription
option. The other alternative to name this new option could be
max_retention_duration. The explanation should clarify that it is used
with the retain_dead_tuples option. I think the other proposed names
appear a bit long to me.
--
With Regards,
Amit Kapila.
On Fri, Aug 22, 2025 at 5:10 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 4, 2025 at 3:11 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 4, 2025 at 11:46 AM shveta malik <shveta.malik@gmail.com> wrote:
7)
Shall we rename 'max_conflict_retention_duration' to
'max_conflict_info_retention_duration' as the latter one is more
clear?Before bikeshedding on the name of this option, I would like us to
once again consider whether we should provide this option at
subscription-level or GUC?Now that we decided that we would like to go with the subscription
option. The other alternative to name this new option could be
max_retention_duration. The explanation should clarify that it is used
with the retain_dead_tuples option. I think the other proposed names
appear a bit long to me.
'max_retention_duration' looks good to me.
thanks
Shveta
On Friday, August 22, 2025 7:17 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Aug 21, 2025 at 2:01 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the V64 patch set addressing this concern.
Few minor comments: 1. static void process_syncing_tables_for_sync(XLogRecPtr current_lsn) { - SpinLockAcquire(&MyLogicalRepWorker->relmutex); + SpinLockAcquire(&MyLogicalRepWorker->mutex);Why is this change part of this patch? Please extract it as a separate
patch unless this change is related to this patch.
Removed these changes for now, will post again once the
main patches get pushed.
2. 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; @@ -1595,6 +1614,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS) elog(ERROR, "unknown worker type"); }+ /* + * Use the worker's oldest_nonremovable_xid instead of + * pg_subscription.subretentionactive to determine whether retention + * is active, as retention resumption might not be complete even when + * subretentionactive is set to true; this is because the launcher + * assigns the initial oldest_nonremovable_xid after the apply worker + * updates the catalog (see resume_conflict_info_retention). + * + * Only the leader apply worker manages conflict retention (see + * maybe_advance_nonremovable_xid() for details). + */ + if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker)) + values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid); + elseThe theory given in the comment sounds good to me but I still suggest
it is better to extract it into a separate patch, so that we can
analyse/test it separately. Also, it will reduce the patch size as
well.
OK, I have moved these changes into the 0003 patch in the
latest version.
3. /* Ensure that we can enable retain_dead_tuples */ if (opts.retaindeadtuples) - CheckSubDeadTupleRetention(true, !opts.enabled, WARNING); + CheckSubDeadTupleRetention(true, !opts.enabled, WARNING, true); + + /* Notify that max_conflict_retention_duration is ineffective */ + else if (opts.maxconflretention) + notify_ineffective_max_conflict_retention(true);Can't we combine these checks by passing both parameters to
CheckSubDeadTupleRetention() and let that function handle all
inappropriate value cases? BTW, even for other places, see if you can
reduce the length of the function name
notify_ineffective_max_conflict_retention.
Attach the V65 patch set which addressed above and
Shveta's comments[1]/messages/by-id/CAJpy0uBFB6K2ZoLebLCBfG+2edu63dU5oS1C6MqcnfcQj4CofQ@mail.gmail.com.
[1]: /messages/by-id/CAJpy0uBFB6K2ZoLebLCBfG+2edu63dU5oS1C6MqcnfcQj4CofQ@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v65-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v65-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From 9280973f522a263b6cc00998a84c2acabb494035 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v65 3/4] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples 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/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..5f93b918274 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index f2e8d6a3057..ccc78d81839 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 d69f78826b4..f9ea1bc2e9f 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1583,7 +1583,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;
@@ -1660,6 +1660,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.51.0.windows.1
v65-0001-Introduce-a-max_conflict_retention_duration-opti.patchapplication/octet-stream; name=v65-0001-Introduce-a-max_conflict_retention_duration-opti.patchDownload
From bd8ec0915c68460b68c6447796b7eaf6cbf33f8c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v65 1/4] Introduce a 'max_conflict_retention_duration' option
to subscriptions.
This commit introduces a subscription option max_conflict_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and 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.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_conflict_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization once at least one apply worker confirms that the retention
duration is within the max_conflict_retention_duration limit.
---
doc/src/sgml/catalogs.sgml | 25 +++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 135 ++++++++++--
src/backend/replication/logical/launcher.c | 71 ++++---
src/backend/replication/logical/worker.c | 236 ++++++++++++++++++++-
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 19 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 6 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 +++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 683 insertions(+), 146 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..98bfa96b17b 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxconflretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..f2c2e147472 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_conflict_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..8de0cd0d53f 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_conflict_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_conflict_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..16eb5c16a0b 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxconflretention = subform->submaxconflretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..f2e8d6a3057 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxconflretention, subretentionactive,
+ 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 4c01d21b2f3..f94c2c6db43 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_CONFLICT_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxconflretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -112,6 +114,7 @@ static void check_publications_origin(WalReceiverConn *wrconn,
Oid *subrel_local_oids, int subrel_count,
char *subname);
static void check_pub_dead_tuple_retention(WalReceiverConn *wrconn);
+static void notify_ineffective_max_retention(bool update_maxretention);
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);
@@ -169,6 +172,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ opts->maxconflretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +328,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_conflict_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_CONFLICT_RETENTION_DURATION;
+ opts->maxconflretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +594,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -647,9 +662,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_conflict_retention_duration is appropriate.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ true, opts.maxconflretention);
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -693,6 +712,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_failover = false;
bool update_two_phase = false;
bool check_pub_rdt = false;
+ bool ineffective_maxconflretention = false;
+ bool update_maxretention = false;
bool retain_dead_tuples;
char *origin;
Subscription *sub;
@@ -1235,7 +1260,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_CONFLICT_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1397,10 +1424,34 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES))
{
+ bool retention_active = sub->retentionactive;
+
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only when there is a change
+ * in the retain_dead_tuples option value.
+ *
+ * It might not be ideal to blindly mark retention as
+ * active upon enabling the retain_dead_tuples, when
+ * retention was previously ceased and the user toggles
+ * retain_dead_tuples without adjusting the publisher
+ * workload. However, since retention will be stopped
+ * again soon in such cases, and this approach offers a
+ * convenient way for the user to manually refresh the
+ * retention status, it is suitable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1421,8 +1472,10 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* Remind the user that enabling subscription will prevent
* the accumulation of dead tuples.
*/
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ opts.retaindeadtuples,
+ retention_active, false,
+ sub->maxconflretention);
/*
* Notify the launcher to manage the replication slot for
@@ -1434,6 +1487,20 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
check_pub_rdt = opts.retaindeadtuples;
retain_dead_tuples = opts.retaindeadtuples;
+
+ ineffective_maxconflretention = (!opts.retaindeadtuples &&
+ sub->maxconflretention);
+ }
+
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_CONFLICT_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxconflretention - 1] =
+ Int32GetDatum(opts.maxconflretention);
+ replaces[Anum_pg_subscription_submaxconflretention - 1] = true;
+
+ update_maxretention = true;
+ ineffective_maxconflretention = (!retain_dead_tuples &&
+ opts.maxconflretention);
}
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
@@ -1453,6 +1520,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
origin = opts.origin;
}
+ if (ineffective_maxconflretention)
+ notify_ineffective_max_retention(update_maxretention);
+
update_tuple = true;
break;
}
@@ -1473,9 +1543,10 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false,
+ sub->maxconflretention);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2468,38 +2539,62 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective (See notify_ineffective_max_retention).
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool check_max_retention, int max_retention)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ if (retain_dead_tuples && check_guc && wal_level < WAL_LEVEL_REPLICA)
ereport(ERROR,
errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
- if (check_guc && !track_commit_timestamp)
+ if (retain_dead_tuples && check_guc && !track_commit_timestamp)
ereport(WARNING,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
errhint("Consider setting \"%s\" to true.",
"track_commit_timestamp"));
- if (sub_disabled)
+ if (retain_dead_tuples && sub_disabled && retention_active)
ereport(elevel_for_sub_disabled,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
(elevel_for_sub_disabled > NOTICE)
? errhint("Consider setting %s to false.",
"retain_dead_tuples") : 0);
+
+ if (!retain_dead_tuples && check_max_retention && max_retention)
+ notify_ineffective_max_retention(true);
+}
+
+/*
+ * Report a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_conflict_retention_duration causes no harm,
+ * even when it is ineffective.
+ */
+static void
+notify_ineffective_max_retention(bool update_maxretention)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ update_maxretention
+ ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled")
+ : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective"));
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..add5fc1ad18 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1183,6 +1186,7 @@ ApplyLauncherMain(Datum main_arg)
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
bool can_advance_xmin = true;
bool retain_dead_tuples = false;
+ bool retention_inactive = false;
TransactionId xmin = InvalidTransactionId;
CHECK_FOR_INTERRUPTS();
@@ -1214,17 +1218,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1233,28 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ can_advance_xmin &= sub->enabled;
+
+ /*
+ * Consider overall retention inactive only when all
+ * subscriptions with retain_dead_tuples enabled have
+ * marked it as inactive.
+ */
+ retention_inactive = false;
+ }
}
if (!sub->enabled)
@@ -1256,7 +1271,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples && sub->retentionactive &&
+ can_advance_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,11 +1281,11 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
+ if (sub->retaindeadtuples && sub->retentionactive)
can_advance_xmin = false;
/*
@@ -1295,7 +1311,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1337,19 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention,
+ * marking it as inactive, the new xmin will be set to
+ * InvalidTransactionId. We then update slot.xmin accordingly to
+ * permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_advance_xmin || retention_inactive)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1402,17 +1425,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..29d0c9a6e45 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,6 +190,16 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
+ * If max_conflict_retention_duration is defined, one additional phase is
+ * involved:
+ *
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * max_conflict_retention_duration. During this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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
@@ -373,7 +383,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +426,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +569,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3220,6 +3237,7 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
{
TransactionId oldestxmin;
ReplicationSlot *slot;
+ bool retention_active;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3228,6 +3246,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
if (!MySubscription->retaindeadtuples || !track_commit_timestamp)
return false;
+ /*
+ * Check whether the leader apply worker has stopped retaining information
+ * for detecting conflicts.
+ *
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention is
+ * active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher assigns
+ * the initial oldest_nonremovable_xid after the apply worker updates the
+ * catalog (see resume_conflict_info_retention).
+ */
+ if (am_leader_apply_worker())
+ {
+ retention_active =
+ TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+ }
+ else
+ {
+ LogicalRepWorker *leader;
+
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
+
+ SpinLockAcquire(&leader->relmutex);
+ retention_active = TransactionIdIsValid(leader->oldest_nonremovable_xid);
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!retention_active)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3315,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4179,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4395,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4424,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4545,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4633,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * Stop retaining conflict information if required (See
+ * should_stop_conflict_info_retention() for details).
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4699,67 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
*/
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxconflretention),
+ errhint("You might need to increase \"%s\".",
+ "max_conflict_retention_duration"));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,9 +4767,56 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
+
+/*
+ * 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, proceed to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Currently, the retention will not resume automatically unless user manually
+ * disables retain_dead_tuples and re-enables it after confirming that the
+ * replication slot has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxconflretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return if the wait time has not exceeded the maximum limit
+ * (max_conflict_retention_duration). The time spent waiting for table
+ * synchronization is not counted, as it's an infrequent operation.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxconflretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
+
+ return true;
}
/*
@@ -4621,8 +4828,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* 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.
+ * The interval is reset to the lesser of 100ms and
+ * max_conflict_retention_duration once there is some activities 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.
@@ -4642,6 +4849,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
}
else
{
@@ -5462,7 +5673,7 @@ InitializeLogRepWorker(void)
* logicalrep_worker_launch.
*/
if (am_leader_apply_worker() &&
- MySubscription->retaindeadtuples &&
+ MySubscription->retaindeadtuples && MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5634,7 +5845,10 @@ DisableSubscriptionAndExit(void)
* context.
*/
if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false,
+ MySubscription->maxconflretention);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..8d8bcf61075 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxconflretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxconflretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxconflretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxconflretention = PQfnumber(res, "submaxconflretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxconflretention =
+ atoi(PQgetvalue(res, i, i_submaxconflretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxconflretention)
+ appendPQExpBuffer(query, ", max_conflict_retention_duration = %d", subinfo->submaxconflretention);
+
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 dde85ed156c..e6b94422af7 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxconflretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..b6d57d02778 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxconflretention AS \"%s\"\n",
+ gettext_noop("Max conflict retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Dead tuple retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..2d7016fe717 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_conflict_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..fcc1ad173ca 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,17 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxconflretention; /* The maximum duration (in
+ * milliseconds) for which information
+ * useful for conflict detection can
+ * be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded
+ * max_conflict_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +147,14 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxconflretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded
+ * max_conflict_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..85f6f45d4fa 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,10 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool check_max_retention,
+ int max_retention);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..b86c759394f 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID would be set to InvalidTransactionId if the apply worker has
+ * stopped retaining information useful for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c6b0784b253 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+ERROR: max_conflict_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+NOTICE: max_conflict_retention_duration has no effect when retain_dead_tuples 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 dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max conflict retention duration | Dead tuple retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+---------------------------------+-----------------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..9b2c489adaf 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_conflict_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_conflict_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_conflict_retention_duration = 0);
+
+\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;
--
2.51.0.windows.1
v65-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v65-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From f10f1ccbb5948266a27890c4e742dff4d77b095d Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 22 Aug 2025 10:09:31 +0800
Subject: [PATCH v65 2/4] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized 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/ref/create_subscription.sgml | 7 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 80 ++++++--
src/backend/replication/logical/worker.c | 204 +++++++++++++++++++--
4 files changed, 267 insertions(+), 33 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 8de0cd0d53f..82fdbe4b7b5 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -540,8 +540,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
retention duration exceeded the
<literal>max_conflict_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
- disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new subscription with retain_dead_tuples
+ enabled is created.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f94c2c6db43..f4411a59418 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -856,7 +856,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add5fc1ad18..d69f78826b4 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_advance_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -1254,6 +1257,13 @@ ApplyLauncherMain(Datum main_arg)
* marked it as inactive.
*/
retention_inactive = false;
+
+ /*
+ * Initialize slot.xmin as a subscription resumes retention
+ * of information useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
}
}
@@ -1271,9 +1281,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && sub->retentionactive &&
- can_advance_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_advance_xmin, &xmin);
/* worker is running already */
continue;
@@ -1385,7 +1394,8 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_advance_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
@@ -1401,7 +1411,27 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_advance_xmin)
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1462,23 +1492,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1498,6 +1520,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 29d0c9a6e45..7b8209ad08f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,7 +190,7 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
- * If max_conflict_retention_duration is defined, one additional phase is
+ * If max_conflict_retention_duration is defined, two additional phases are
* involved:
*
* - RDT_STOP_CONFLICT_INFO_RETENTION:
@@ -200,6 +200,20 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_conflict_retention_duration is set to 0). Additionally, it
+ * initiates if conditions improve allowing more efficient advancement. During
+ * this phase, pg_subscription.subretentionactive is updated to true within a
+ * new transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ *
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION at
+ * any phase if the retention has been stopped, but
+ * max_conflict_retention_duration is now set to 0.
+ *
* 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
@@ -385,6 +399,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -429,6 +444,10 @@ typedef struct RetainDeadTuplesData
long table_sync_wait_time; /* time spent waiting for table sync
* to finish */
+ bool wait_for_initial_xid; /* wait for the launcher to initialize
+ * the apply worker's
+ * oldest_nonremovable_xid */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -572,6 +591,9 @@ static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4373,6 +4395,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4395,10 +4424,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4427,6 +4452,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4683,6 +4711,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4749,6 +4789,109 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
ApplyLauncherWakeup();
reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ if (rdt_data->wait_for_initial_xid)
+ {
+ TransactionId nonremovable_xid;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Proceed to the next phase if the catalog has been updated and the
+ * launcher has initialized slot.xmin and assigned it to
+ * oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race
+ * condition where slot.xmin is set to InvalidTransactionId
+ * immediately after the check. In such cases, oldest_nonremovable_xid
+ * would no longer be protected by a replication slot and could become
+ * unreliable if a wraparound occurs.
+ */
+ if (TransactionIdIsValid(nonremovable_xid))
+ {
+ rdt_data->wait_for_initial_xid = false;
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ /*
+ * Return early since the catalog has been updated and we are waiting
+ * for oldest_nonremovable_xid to be initialized.
+ */
+ return;
+ }
+
+ /*
+ * Proceed to the next phase if retention has not been stopped yet. This
+ * occurs when transitioning from the RDT_STOP_CONFLICT_INFO_RETENTION
+ * phase but subretentionactive has not been updated due to the inability
+ * to start a new transaction (see stop_conflict_info_retention).
+ */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return;
+ }
+
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to true */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, true);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxconflretention
+ ? errdetail("The retention duration for information used in conflict detection is now within the acceptable limit of %u ms.",
+ MySubscription->maxconflretention)
+ : errdetail("The retention duration for information used in conflict detection is now indefinite."));
+
+ /* Notify launcher to update the xmin of the conflict slot */
+ ApplyLauncherWakeup();
+
+ /*
+ * The next step is to wait for the launcher to initialize the
+ * oldest_nonremovable_xid.
+ */
+ rdt_data->wait_for_initial_xid = true;
}
/*
@@ -4778,9 +4921,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples 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(RetainDeadTuplesData *rdt_data)
@@ -4811,14 +4953,54 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_conflict_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_conflict_retention_duration is not 0 */
+ if (MySubscription->maxconflretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.51.0.windows.1
On Mon, Aug 25, 2025 at 10:06 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V65 patch set which addressed above and
Shveta's comments[1].
Thank You for the patches, please find a few comments on v64 itself (I
think valid on v65 as well):
1) in resume_conflict_info_retention(), shall we rewrite the code as:
resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
{
if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
Assert(MySubscription->retentionactive);
reset_retention_data_fields(rdt_data);
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
return;
}
--rest of the code to start a transaction and update the catalog to
set retentionactive=true.
}
When 'MyLogicalRepWorker->oldest_nonremovable_xid' is valid, it either
means retention has not been stopped yet due to the inability to start
a txn or it is that stage of resume (subsequent calls) where catalog
has been updated and the launcher has initialized slot.xmin and
assigned it to oldest_nonremovable_xid. In both cases
'retentionactive' should be true by now and thus the Assert can be put
(as above). The dependency on 'wait_for_initial_xid' can be removed if
we do it like this.
2)
Also, I feel the code part which does txn and snapshot processing
(Start, Push, Pop, Commit, launcher-wakeup) and update catalog can be
shifted to one function and both stop_conflict_info_retention() and
resume_conflict_info_retention() can call it.
thanks
Shveta
On Mon, Aug 25, 2025 at 12:09 PM shveta malik <shveta.malik@gmail.com> wrote:
On Mon, Aug 25, 2025 at 10:06 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Attach the V65 patch set which addressed above and
Shveta's comments[1].Thank You for the patches, please find a few comments on v64 itself (I
think valid on v65 as well):
All previously reported bugs seem to be fixed on the latest patch.
Please find a few comments on v65 though:
1)
The comment atop adjust_xid_advance_interval() says:
~~
* The interval is reset to the lesser of 100ms and
* max_conflict_retention_duration once there is some activities on the node.
~~
But we are not doing min of the 2 values when we find new-xid (i.e.
we see activity on node)
/*
* A new transaction ID was found or the interval is not yet
* initialized, so set the interval to the minimum value.
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
2)
* max_conflict_retention_duration once there is some activities on the node.
activities-->activity
3)
On putting some logs, when all subs stopped retention, the values are:
LOG: ***** can_advance_xmin:1, retention_inactive:0, xmin:0
LOG: ***** can_advance_xmin:1, retention_inactive:0, xmin:0
can_advance_xmin:1 and xmin=0 seems contradictory. Can we do something
here to improve this situation?
4)
Also I fail to think of a scenario where retention_inactive will be
useful as everything seems to be handled using can_advance_xmin
already i.e. slot is updated to have invalid xmin only through
'can_advance_xmin' without relying on retention_inactive.
thanks
Shveta
On Mon, Aug 25, 2025 at 10:06 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Attach the V65 patch set which addressed above and
Shveta's comments[1].
A few comments on 0001:
1.
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ opts.retaindeadtuples,
+ retention_active, false,
+ sub->maxconflretention);
/*
* Notify the launcher to manage the replication slot for
@@ -1434,6 +1487,20 @@ AlterSubscription(ParseState *pstate,
AlterSubscriptionStmt *stmt,
check_pub_rdt = opts.retaindeadtuples;
retain_dead_tuples = opts.retaindeadtuples;
+
+ ineffective_maxconflretention = (!opts.retaindeadtuples &&
+ sub->maxconflretention);
Why can't we handle this special ineffective_maxconflretention case
inside CheckSubDeadTupleRetention? If so, then we can directly give
the NOTICE in case of SUBOPT_MAX_CONFLICT_RETENTION_DURATION without
having a separate notify_ineffective_max_retention() function.
2.
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples && sub->retentionactive &&
+ can_advance_xmin)
This coding pattern looks odd, you can have one condition per line.
3. Are we setting retention_inactive in launcher.c to true ever?
4.
this is because the launcher assigns
+ * the initial oldest_nonremovable_xid after the apply worker updates the
+ * catalog (see resume_conflict_info_retention).
I don't see resume_conflict_info_retention in 0001, so I couldn't make
sense of this part of the comment.
--
With Regards,
Amit Kapila.
On Mon, Aug 25, 2025 at 5:05 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
A few comments on 0001:
Some more comments:
1.
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!retention_active)
+ return false;
+
/*
* For conflict detection, we use the conflict slot's xmin value instead
* of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
@@ -3254,7 +3315,15 @@ FindDeletedTupleInLocalRel(Relation localrel,
Oid localidxoid,
oldestxmin = slot->data.xmin;
SpinLockRelease(&slot->mutex);
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the conflict detection slot.xmin is set to
+ * InvalidTransactionId. This situation arises if the current worker is
+ * either a table synchronization or parallel apply worker, and the leader
+ * stopped retention immediately after checking the
+ * oldest_nonremovable_xid above.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
If the current worker is tablesync or parallel_apply, it should have
exited from the above check of retention_active as we get the leader's
oldest_nonremovable_xid to decide that. What am, I missing? This made
me wonder whether we need to use slot's xmin after we have fetched
leader's oldest_nonremovable_xid to find deleted tuple?
2.
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * The interval is reset to the lesser of 100ms and
+ * max_conflict_retention_duration once there is some activities on the node.
AFAICS, this is not adhered in the code because you are using it when
there is no activity aka when new_xid_found is false. IS the comment
wrong or code needs some updation?
3.
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxconflretention);
Can't we combine it with calculation of max_interval few lines above
this change? And also adjust comments atop
adjust_xid_advance_interval() accordingly?
4.
if (am_leader_apply_worker() &&
- MySubscription->retaindeadtuples &&
+ MySubscription->retaindeadtuples && MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
I think this code can look neat if you have one condition per line.
Apart from above comments, I have tried to improve some code comments
in the attached.
--
With Regards,
Amit Kapila.
Attachments:
v65_amit_1.patch.txttext/plain; charset=US-ASCII; name=v65_amit_1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 29d0c9a6e45..3df1828e755 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4546,8 +4546,8 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
return;
/*
- * Stop retaining conflict information if required (See
- * should_stop_conflict_info_retention() for details).
+ * We don't need to maintain oldest_nonremovable_xid if we decide
+ * to stop retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
return;
@@ -4650,8 +4650,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
}
/*
- * Stop retaining conflict information if required (See
- * should_stop_conflict_info_retention() for details).
+ * We don't need to maintain oldest_nonremovable_xid if we decide
+ * to stop retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
return;
@@ -4771,16 +4771,16 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
}
/*
- * Check whether conflict information retention should be stopped because the
- * wait time has exceeded the maximum limit (max_conflict_retention_duration).
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_conflict_retention_duration).
*
- * If retention should be stopped, proceed to the
+ * If retention should be stopped, transition to the
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples and re-enables it after confirming that the
- * replication slot has been dropped.
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4802,9 +4802,10 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
now = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
/*
- * Return if the wait time has not exceeded the maximum limit
- * (max_conflict_retention_duration). The time spent waiting for table
- * synchronization is not counted, as it's an infrequent operation.
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_conflict_retention_duration). Time spent waiting for table
+ * synchronization is excluded from this calculation, as it occurs
+ * infrequently.
*/
if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
MySubscription->maxconflretention +
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b86c759394f..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -95,8 +95,8 @@ typedef struct LogicalRepWorker
* named "pg_conflict_detection". It asynchronously collects this ID to
* decide when to advance the xmin value of the slot.
*
- * This ID would be set to InvalidTransactionId if the apply worker has
- * stopped retaining information useful for conflict detection.
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
On Tue, Aug 26, 2025 at 12:15 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 25, 2025 at 5:05 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
Some comments on latest patch
0001:
1.
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially missing some conflict detections.
+ </para>
We can improve this wording by saying "potentially incorrectly
detecting some conflict"
2.
@@ -1175,6 +1198,8 @@ AlterSubscription(ParseState *pstate,
AlterSubscriptionStmt *stmt,
bool update_failover = false;
bool update_two_phase = false;
bool check_pub_rdt = false;
+ bool ineffective_maxconflretention = false;
+ bool update_maxretention = false;
For making variable names more consistent, better to change
'ineffective_maxconflretention' to 'ineffective_maxretention' so that
this will be more consistent with 'update_maxretention'
3.
+/*
+ * Report a NOTICE to inform users that max_conflict_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_conflict_retention_duration
causes no harm,
+ * even when it is ineffective.
+ */
+static void
+notify_ineffective_max_retention(bool update_maxretention)
+{
+ ereport(NOTICE,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ update_maxretention
+ ? errmsg("max_conflict_retention_duration has no effect when
retain_dead_tuples is disabled")
+ : errmsg("disabling retain_dead_tuples will render
max_conflict_retention_duration ineffective"));
}
I really don't like to make a function for a single ereport, even if
this is being called from multiple places.
--
Regards,
Dilip Kumar
Google
Please find some more comments:
1)
In CheckSubDeadTupleRetention(), shall we have below instead of
retain_dead_tuples check in all conditions?
if (retain_dead_tuples)
guc checks (wal_level and tracl_commit)
else
max retention check
2)
Currently stop and resume messages are:
~~
LOG: logical replication worker for subscription "sub2" has stopped
retaining the information for detecting conflicts
DETAIL: The retention duration for information used in conflict
detection has exceeded the maximum limit of 10000 ms.
HINT: You might need to increase "max_conflict_retention_duration".
--
LOG: logical replication worker for subscription "sub2" will resume
retaining the information for detecting conflicts
DETAIL: The retention duration for information used in conflict
detection is now within the acceptable limit of 10000 ms.
~~
Resume message does not mention GUC while stop does mention it in
HINT. Shall we have both stop and resume DETAIL msg mention GUC as:
Stop:
DETAIL: Retention of information used for conflict detection has
exceeded max_conflict_retention_duration of 10000 ms.
Resume:
DETAIL: Retention of information used for conflict detection is now
within the max_conflict_retention_duration of 1000 ms.
I think we should get rid of HINT in stop msg as that is not what we
actually should be suggesting without knowing the
system-workload/bloat condition. Hint seems oversimplified and thus
incomplete looking at the possibilities we may have here.
3)
CREATE SUBSCRIPTION sub CONNECTION '...' PUBLICATION pub WITH (connect
= false, retain_dead_tuples = true, max_conflict_retention_duration =
5000);
WARNING: deleted rows to detect conflicts would not be removed until
the subscription is enabled
HINT: Consider setting retain_dead_tuples to false.
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.
CREATE SUBSCRIPTION
With connect=false, we get above messages. Reverse order of WARNINGs
will make more sense as 'not connected' WARNING and HINT clarifies a
few things including that the sub is disabled and needs to be enabled.
Can we attempt doing it provided it does not over-complicate code?
4)
postgres=# \dRs+
List of subscriptions
.. | Retain dead tuples | Max conflict retention duration | Dead tuple
retention active |..
Here shall we have 'Max retention duration' and 'Retention Active'
instead of 'Max conflict retention duration' and 'Dead tuple retention
active'?
thanks
Shveta
On Tuesday, August 26, 2025 5:02 PM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Tue, Aug 26, 2025 at 12:15 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:On Mon, Aug 25, 2025 at 5:05 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
Some comments on latest patch
Thanks for the comments!
0001:
1. + <para> + Note that setting a non-zero value for this option could lead to + information for conflict detection being removed prematurely, + potentially missing some conflict detections. + </para>We can improve this wording by saying "potentially incorrectly detecting some
conflict"
I slightly reworded it to "potentially resulting in incorrect conflict detection."
2. @@ -1175,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool update_failover = false; bool update_two_phase = false; bool check_pub_rdt = false; + bool ineffective_maxconflretention = false; bool update_maxretention = + false;For making variable names more consistent, better to change
'ineffective_maxconflretention' to 'ineffective_maxretention' so that this will be
more consistent with 'update_maxretention'3. +/* + * Report a NOTICE to inform users that max_conflict_retention_duration +is + * ineffective when retain_dead_tuples is disabled for a subscription. +An ERROR + * is not issued because setting max_conflict_retention_duration causes no harm, + * even when it is ineffective. + */ +static void +notify_ineffective_max_retention(bool update_maxretention) { +ereport(NOTICE, errcode(ERRCODE_INVALID_PARAMETER_VALUE), + update_maxretention + ? errmsg("max_conflict_retention_duration has no effect when retain_dead_tuples is disabled") + : errmsg("disabling retain_dead_tuples will render max_conflict_retention_duration ineffective")); }I really don't like to make a function for a single ereport, even if this is being
called from multiple places.
I refactored this part based on some other comments, so these points
is addressed in the V66 patch set as well.
Best Regards,
Hou zj
Attachments:
v66-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v66-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From 5fed4ef615c690258f68da3f13ea3c9e0d7900fa Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v66 3/3] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fbd3d1900f4..4679cb70213 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 73d2c22ae8b..6d3559038bd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1575,7 +1575,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;
@@ -1652,6 +1652,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.31.1
v66-0001-Introduce-a-max_retention_duration-option-to-sub.patchapplication/octet-stream; name=v66-0001-Introduce-a-max_retention_duration-option-to-sub.patchDownload
From a275ccc7b232218ff62b5f445813562d9d4fa524 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v66 1/3] Introduce a 'max_retention_duration' option to
subscriptions.
This commit introduces a subscription option max_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and the apply worker cannot catch up with
the publisher's workload.
If the time spent advancing non-removable transaction ID surpasses the
max_retention_duration threshold, the apply worker would stop retaining
information for conflict detection. The replication slot
pg_conflict_detection.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization once at least one apply worker confirms that the retention
duration is within the max_retention_duration limit.
---
doc/src/sgml/catalogs.sgml | 25 +++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 162 +++++++++----
src/backend/replication/logical/launcher.c | 65 +++---
src/backend/replication/logical/worker.c | 250 ++++++++++++++++++---
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 16 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 5 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 665 insertions(+), 195 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..89f2eeb601e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..25105556d7a 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..478b8352164 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially resulting in incorrect conflict detection.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..b885890de37 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxretention = subform->submaxretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..fbd3d1900f4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxretention, subretentionactive,
+ 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 4c01d21b2f3..f9d92702434 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION))
+ opts->maxretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_RETENTION_DURATION;
+ opts->maxretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -647,9 +661,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_retention_duration is appropriate.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ (opts.maxretention > 0));
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -693,6 +711,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1176,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_two_phase = false;
bool check_pub_rdt = false;
bool retain_dead_tuples;
+ int max_retention;
+ bool retention_active;
char *origin;
Subscription *sub;
Form_pg_subscription form;
@@ -1206,6 +1230,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = sub->retaindeadtuples;
origin = sub->origin;
+ max_retention = sub->maxretention;
+ retention_active = sub->retentionactive;
/*
* Don't allow non-superuser modification of a subscription with
@@ -1235,7 +1261,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1401,6 +1429,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only if there's a change in
+ * the retain_dead_tuples option value.
+ *
+ * Automatically marking retention as active when
+ * retain_dead_tuples is enabled may not always be ideal,
+ * especially if retention was previously stopped and the
+ * user toggles retain_dead_tuples without adjusting the
+ * publisher workload. However, this behavior provides a
+ * convenient way for users to manually refresh the
+ * retention status. Since retention will be stopped again
+ * unless the publisher workload is reduced, this approach
+ * is acceptable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1417,13 +1468,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
errhint("Try again after some time.")));
- /*
- * Remind the user that enabling subscription will prevent
- * the accumulation of dead tuples.
- */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
-
/*
* Notify the launcher to manage the replication slot for
* conflict detection. This ensures that replication slot
@@ -1436,6 +1480,26 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ replaces[Anum_pg_subscription_submaxretention - 1] = true;
+
+ max_retention = opts.maxretention;
+ }
+
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_retention_duration is appropriate.
+ */
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
+ IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retain_dead_tuples,
+ retention_active,
+ (max_retention > 0));
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1473,9 +1537,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2468,38 +2532,54 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_retention_duration causes no harm,
+ * even when it is ineffective.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool max_retention_set)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
- ereport(ERROR,
- errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
- errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
-
- if (check_guc && !track_commit_timestamp)
- ereport(WARNING,
- errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
- errhint("Consider setting \"%s\" to true.",
- "track_commit_timestamp"));
-
- if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ if (retain_dead_tuples)
+ {
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled && retention_active)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+ }
+ else if (max_retention_set)
+ {
+ ereport(NOTICE,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_dead_tuples") : 0);
+ errmsg("max_retention_duration is ineffective when retain_dead_tuples is disabled"));
+ }
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..c0211867881 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1181,7 +1184,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool can_advance_xmin = true;
+ bool can_update_xmin = true;
bool retain_dead_tuples = false;
TransactionId xmin = InvalidTransactionId;
@@ -1214,17 +1217,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1232,18 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions
+ * actively retaining dead tuples are enabled. This is
+ * required to ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ if (sub->retentionactive)
+ can_update_xmin &= sub->enabled;
}
if (!sub->enabled)
@@ -1256,7 +1260,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,12 +1271,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
- can_advance_xmin = false;
+ if (sub->retaindeadtuples && sub->retentionactive)
+ can_update_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1295,7 +1301,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1327,19 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention,
+ * marking it as inactive, the new xmin will be set to
+ * InvalidTransactionId. We then update slot.xmin accordingly to
+ * permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_update_xmin)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1402,17 +1415,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..d378eb08c71 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,6 +190,16 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
+ * If max_retention_duration is defined, one additional phase is
+ * involved:
+ *
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * max_retention_duration. During this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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
@@ -373,7 +383,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +426,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +569,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3219,7 +3236,6 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
TimestampTz *delete_time)
{
TransactionId oldestxmin;
- ReplicationSlot *slot;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3229,32 +3245,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
return false;
/*
- * For conflict detection, we use the conflict slot's xmin value instead
- * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
- * a threshold to identify tuples that were recently deleted. These tuples
- * are not visible to concurrent transactions, but we log an
- * update_deleted conflict if such a tuple matches the remote update being
- * applied.
- *
- * Although GetOldestNonRemovableTransactionId() can return a value older
- * than the slot's xmin, for our current purpose it is acceptable to treat
- * tuples deleted by transactions prior to slot.xmin as update_missing
- * conflicts.
+ * For conflict detection, we use the leader worker's
+ * oldest_nonremovable_xid value instead of invoking
+ * GetOldestNonRemovableTransactionId() or relying on the conflict
+ * detection slot's xmin. The oldest_nonremovable_xid acts as a threshold
+ * to identify tuples that were recently deleted. These tuples are not
+ * visible to concurrent transactions, but we log an update_deleted
+ * conflict if such a tuple matches the remote update being applied.
*
- * Ideally, we would use oldest_nonremovable_xid, which is directly
- * maintained by the leader apply worker. However, this value is not
- * available to table synchronization or parallel apply workers, making
- * slot.xmin a practical alternative in those contexts.
+ * Although GetOldestNonRemovableTransactionId() and slot.xmin can return
+ * a value older than the oldest_nonremovable_xid, for our current purpose
+ * it is acceptable to treat tuples deleted by transactions prior to
+ * oldest_nonremovable_xid as update_missing conflicts.
*/
- slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+ if (am_leader_apply_worker())
+ {
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+ }
+ else
+ {
+ LogicalRepWorker *leader;
- Assert(slot);
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
- SpinLockAcquire(&slot->mutex);
- oldestxmin = slot->data.xmin;
- SpinLockRelease(&slot->mutex);
+ SpinLockAcquire(&leader->relmutex);
+ oldestxmin = leader->oldest_nonremovable_xid;
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4143,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4359,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4388,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4509,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4597,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4663,65 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
*/
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,22 +4729,70 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
+
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
+
+ return true;
}
/*
* 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.
+ * 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. However, this maximum interval will not exceed
+ * max_retention_duration.
*
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * The interval is reset to the lesser of 100ms and
+ * max_retention_duration 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.
@@ -4651,6 +4821,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
@@ -5463,6 +5637,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5633,8 +5808,9 @@ DisableSubscriptionAndExit(void)
* an error, as verifying commit timestamps is unnecessary in this
* context.
*/
- if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..bea793456f9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxretention = PQfnumber(res, "submaxretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxretention =
+ atoi(PQgetvalue(res, i, i_submaxretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxretention)
+ appendPQExpBuffer(query, ", max_retention_duration = %d", subinfo->submaxretention);
+
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 dde85ed156c..bcc94ff07cc 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..4aa793d7de7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxretention AS \"%s\"\n",
+ gettext_noop("Max retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..6b20a4404b2 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..55cb9b1eefa 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,15 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +145,13 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxretention; /* The maximum duration (in milliseconds) for
+ * which information useful for conflict
+ * detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..fb4e26a51a4 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,9 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool max_retention_set);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c7f1266fc2f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+ERROR: max_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+NOTICE: max_retention_duration is ineffective when retain_dead_tuples 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 dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..ef0c298d2df 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+
+\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;
--
2.31.1
v66-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v66-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From e2db073ab818d56e9fa587549674603265a3132f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 26 Aug 2025 18:55:57 +0800
Subject: [PATCH v66 2/3] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized invalidated slot, if at
least one apply worker has confirmed that the retention duration is now within
the max_retention_duration.
---
doc/src/sgml/ref/create_subscription.sgml | 7 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 102 ++++++++---
src/backend/replication/logical/worker.c | 200 ++++++++++++++++++---
4 files changed, 261 insertions(+), 57 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 478b8352164..2d2800a8b9a 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -540,8 +540,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
retention duration exceeded the
<literal>max_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
- disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new subscription with retain_dead_tuples
+ enabled is created.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f9d92702434..5e737f97766 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -855,7 +855,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0211867881..73d2c22ae8b 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -1233,17 +1236,27 @@ ApplyLauncherMain(Datum main_arg)
*/
CreateConflictDetectionSlot();
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * actively retaining dead tuples are enabled. This is
- * required to ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
can_update_xmin &= sub->enabled;
+
+ /*
+ * Initialize slot.xmin as a subscription resumes
+ * retention of information useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+ }
}
if (!sub->enabled)
@@ -1260,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1375,7 +1386,8 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
@@ -1391,7 +1403,27 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_update_xmin)
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1452,23 +1484,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1488,6 +1512,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d378eb08c71..bc552ff036c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,7 +190,7 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
- * If max_retention_duration is defined, one additional phase is
+ * If max_retention_duration is defined, two additional phases are
* involved:
*
* - RDT_STOP_CONFLICT_INFO_RETENTION:
@@ -200,6 +200,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ *
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION at
+ * any phase if the retention has been stopped, but
+ * max_retention_duration is now set to 0.
+ *
* 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
@@ -385,6 +398,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -572,6 +586,10 @@ static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4337,6 +4355,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4359,10 +4384,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4391,6 +4412,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4647,6 +4671,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4674,6 +4710,91 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxretention),
+ errhint("You might need to increase \"%s\".",
+ "max_retention_duration"));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_conflict_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4691,26 +4812,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
@@ -4740,9 +4851,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4773,14 +4883,54 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.31.1
On Tuesday, August 26, 2025 2:45 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Aug 25, 2025 at 5:05 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:A few comments on 0001:
Some more comments:
Thanks for the comments!
1. + /* + * Return false if the leader apply worker has stopped retaining + * information for detecting conflicts. This implies that + update_deleted + * can no longer be reliably detected. + */ + if (!retention_active) + return false; + /* * For conflict detection, we use the conflict slot's xmin value instead * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as @@ -3254,7 +3315,15 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid, oldestxmin = slot->data.xmin; SpinLockRelease(&slot->mutex);- Assert(TransactionIdIsValid(oldestxmin)); + /* + * Return false if the conflict detection slot.xmin is set to + * InvalidTransactionId. This situation arises if the current worker is + * either a table synchronization or parallel apply worker, and the + leader + * stopped retention immediately after checking the + * oldest_nonremovable_xid above. + */ + if (!TransactionIdIsValid(oldestxmin)) + return false;If the current worker is tablesync or parallel_apply, it should have exited from
the above check of retention_active as we get the leader's
oldest_nonremovable_xid to decide that. What am, I missing? This made me
wonder whether we need to use slot's xmin after we have fetched leader's
oldest_nonremovable_xid to find deleted tuple?
There was a race condition that the leader could stop retention immediately
after the pa worker fetches its oldest_nonremovable_xid. But I agree that it
would be simpler to directly use oldest_nonremovable_xid to find deleted tuple,
so changed. This way, the logic can be simplified.
2. - * The interval is reset to a minimum value of 100ms once there is some - * activity on the node. + * The interval is reset to the lesser of 100ms and + * max_conflict_retention_duration once there is some activities on the node. AFAICS, this is not adhered in the code because you are using it when there is no activity aka when new_xid_found is false. IS the comment wrong or code needs some updation?
I think both needs to be updated. I have adjusted the code to consider
max_retention in both cases and update the comments.
All the comments have been addressed in V66 patch set.
Best Regards,
Hou zj
On Tuesday, August 26, 2025 6:45 PM shveta malik <shveta.malik@gmail.com> wrote:
Please find some more comments:
Thanks for the comments!
3)
CREATE SUBSCRIPTION sub CONNECTION '...' PUBLICATION pub WITH
(connect = false, retain_dead_tuples = true, max_conflict_retention_duration
= 5000);
WARNING: deleted rows to detect conflicts would not be removed until the
subscription is enabled
HINT: Consider setting retain_dead_tuples to false.
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.
CREATE SUBSCRIPTIONWith connect=false, we get above messages. Reverse order of WARNINGs will
make more sense as 'not connected' WARNING and HINT clarifies a few things
including that the sub is disabled and needs to be enabled.
Can we attempt doing it provided it does not over-complicate code?
I agree it makes sense for reversing the messages. But since this behavior isn't
caused by the remaining patches, we can consider it a separate improvement after
the main patch is pushed.
All other comments look good to me, so addressed in the latest patches.
Here is V66 patch set which includes the following changes:
[0001]: * Enhanced documentation according to Dilip's comments [1]. * Simplified logic by directly using the leader's oldest_nonremovable_xid to locate deleted tuples, according to Amit's comments [2]. * Merged Amit's diff [2]. * Integrated the new NOTICE into the existing CheckSubDeadTupleRetention function according to Amit's feedback [3]. * Some other adjustments according to feedback from [2] [3].
* Enhanced documentation according to Dilip's comments [1]/messages/by-id/CAFiTN-v2-Jv3UFYQ48pbX+jb+MXWoxrfsRXQ_J1s1xqPq8P_zg@mail.gmail.com.
* Simplified logic by directly using the leader's oldest_nonremovable_xid to
locate deleted tuples, according to Amit's comments [2]/messages/by-id/CAA4eK1+HcrkKfXAwEsXK0waDK8VSx1qjBVj95SmZKPM0vMF=Qg@mail.gmail.com.
* Merged Amit's diff [2]/messages/by-id/CAA4eK1+HcrkKfXAwEsXK0waDK8VSx1qjBVj95SmZKPM0vMF=Qg@mail.gmail.com.
* Integrated the new NOTICE into the existing CheckSubDeadTupleRetention
function according to Amit's feedback [3]/messages/by-id/CAA4eK1JhYwJhU4vYPGeh8Y46S+FS5ciATw5beJKPrkF5ZAu2AQ@mail.gmail.com.
* Some other adjustments according to feedback from [2]/messages/by-id/CAA4eK1+HcrkKfXAwEsXK0waDK8VSx1qjBVj95SmZKPM0vMF=Qg@mail.gmail.com [3]/messages/by-id/CAA4eK1JhYwJhU4vYPGeh8Y46S+FS5ciATw5beJKPrkF5ZAu2AQ@mail.gmail.com.
[0002]: * Refactored code according to Shveta's comments [4]. * Removed an unnecessary variable according to Shveta's suggestions [5]. * Some other adjustments according to feedback from [4][5][6].
* Refactored code according to Shveta's comments [4]/messages/by-id/CAJpy0uCrHtwN3wgnC26G8M4jQfaMJG3EUU3OY+zpwQPeifjmTg@mail.gmail.com.
* Removed an unnecessary variable according to Shveta's suggestions [5]/messages/by-id/CAJpy0uDpX6jQWC3-cyA38ANT0L_L_qQeWPy2cATzSpLNDha1=A@mail.gmail.com.
* Some other adjustments according to feedback from [4]/messages/by-id/CAJpy0uCrHtwN3wgnC26G8M4jQfaMJG3EUU3OY+zpwQPeifjmTg@mail.gmail.com[5]/messages/by-id/CAJpy0uDpX6jQWC3-cyA38ANT0L_L_qQeWPy2cATzSpLNDha1=A@mail.gmail.com[6]/messages/by-id/CAJpy0uBQ_v0D3ceuZfJrx=zH6-59ORLqj+aqZJ7AQnw3vRRcSA@mail.gmail.com.
[1]: /messages/by-id/CAFiTN-v2-Jv3UFYQ48pbX+jb+MXWoxrfsRXQ_J1s1xqPq8P_zg@mail.gmail.com
[2]: /messages/by-id/CAA4eK1+HcrkKfXAwEsXK0waDK8VSx1qjBVj95SmZKPM0vMF=Qg@mail.gmail.com
[3]: /messages/by-id/CAA4eK1JhYwJhU4vYPGeh8Y46S+FS5ciATw5beJKPrkF5ZAu2AQ@mail.gmail.com
[4]: /messages/by-id/CAJpy0uCrHtwN3wgnC26G8M4jQfaMJG3EUU3OY+zpwQPeifjmTg@mail.gmail.com
[5]: /messages/by-id/CAJpy0uDpX6jQWC3-cyA38ANT0L_L_qQeWPy2cATzSpLNDha1=A@mail.gmail.com
[6]: /messages/by-id/CAJpy0uBQ_v0D3ceuZfJrx=zH6-59ORLqj+aqZJ7AQnw3vRRcSA@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v66-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v66-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From 5fed4ef615c690258f68da3f13ea3c9e0d7900fa Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v66 3/3] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fbd3d1900f4..4679cb70213 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 73d2c22ae8b..6d3559038bd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1575,7 +1575,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;
@@ -1652,6 +1652,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 36aeb14c563..c1b8ede81cb 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.31.1
v66-0001-Introduce-a-max_retention_duration-option-to-sub.patchapplication/octet-stream; name=v66-0001-Introduce-a-max_retention_duration-option-to-sub.patchDownload
From a275ccc7b232218ff62b5f445813562d9d4fa524 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v66 1/3] Introduce a 'max_retention_duration' option to
subscriptions.
This commit introduces a subscription option max_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and the apply worker cannot catch up with
the publisher's workload.
If the time spent advancing non-removable transaction ID surpasses the
max_retention_duration threshold, the apply worker would stop retaining
information for conflict detection. The replication slot
pg_conflict_detection.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization once at least one apply worker confirms that the retention
duration is within the max_retention_duration limit.
---
doc/src/sgml/catalogs.sgml | 25 +++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 162 +++++++++----
src/backend/replication/logical/launcher.c | 65 +++---
src/backend/replication/logical/worker.c | 250 ++++++++++++++++++---
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 16 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 5 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 665 insertions(+), 195 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..89f2eeb601e 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..25105556d7a 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-conflict-retention-duration"><literal>max_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..478b8352164 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-conflict-retention-duration">
+ <term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially resulting in incorrect conflict detection.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..b885890de37 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxretention = subform->submaxretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..fbd3d1900f4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxretention, subretentionactive,
+ 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 4c01d21b2f3..f9d92702434 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION))
+ opts->maxretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_RETENTION_DURATION;
+ opts->maxretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -647,9 +661,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_retention_duration is appropriate.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ (opts.maxretention > 0));
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -693,6 +711,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1176,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_two_phase = false;
bool check_pub_rdt = false;
bool retain_dead_tuples;
+ int max_retention;
+ bool retention_active;
char *origin;
Subscription *sub;
Form_pg_subscription form;
@@ -1206,6 +1230,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = sub->retaindeadtuples;
origin = sub->origin;
+ max_retention = sub->maxretention;
+ retention_active = sub->retentionactive;
/*
* Don't allow non-superuser modification of a subscription with
@@ -1235,7 +1261,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1401,6 +1429,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only if there's a change in
+ * the retain_dead_tuples option value.
+ *
+ * Automatically marking retention as active when
+ * retain_dead_tuples is enabled may not always be ideal,
+ * especially if retention was previously stopped and the
+ * user toggles retain_dead_tuples without adjusting the
+ * publisher workload. However, this behavior provides a
+ * convenient way for users to manually refresh the
+ * retention status. Since retention will be stopped again
+ * unless the publisher workload is reduced, this approach
+ * is acceptable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1417,13 +1468,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
errhint("Try again after some time.")));
- /*
- * Remind the user that enabling subscription will prevent
- * the accumulation of dead tuples.
- */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
-
/*
* Notify the launcher to manage the replication slot for
* conflict detection. This ensures that replication slot
@@ -1436,6 +1480,26 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ replaces[Anum_pg_subscription_submaxretention - 1] = true;
+
+ max_retention = opts.maxretention;
+ }
+
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_retention_duration is appropriate.
+ */
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
+ IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retain_dead_tuples,
+ retention_active,
+ (max_retention > 0));
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1473,9 +1537,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2468,38 +2532,54 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_retention_duration causes no harm,
+ * even when it is ineffective.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool max_retention_set)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
- ereport(ERROR,
- errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
- errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
-
- if (check_guc && !track_commit_timestamp)
- ereport(WARNING,
- errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
- errhint("Consider setting \"%s\" to true.",
- "track_commit_timestamp"));
-
- if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ if (retain_dead_tuples)
+ {
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled && retention_active)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+ }
+ else if (max_retention_set)
+ {
+ ereport(NOTICE,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_dead_tuples") : 0);
+ errmsg("max_retention_duration is ineffective when retain_dead_tuples is disabled"));
+ }
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..c0211867881 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1181,7 +1184,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool can_advance_xmin = true;
+ bool can_update_xmin = true;
bool retain_dead_tuples = false;
TransactionId xmin = InvalidTransactionId;
@@ -1214,17 +1217,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1232,18 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions
+ * actively retaining dead tuples are enabled. This is
+ * required to ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ if (sub->retentionactive)
+ can_update_xmin &= sub->enabled;
}
if (!sub->enabled)
@@ -1256,7 +1260,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,12 +1271,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
- can_advance_xmin = false;
+ if (sub->retaindeadtuples && sub->retentionactive)
+ can_update_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1295,7 +1301,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1327,19 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention,
+ * marking it as inactive, the new xmin will be set to
+ * InvalidTransactionId. We then update slot.xmin accordingly to
+ * permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_update_xmin)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1402,17 +1415,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..d378eb08c71 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,6 +190,16 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
+ * If max_retention_duration is defined, one additional phase is
+ * involved:
+ *
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * max_retention_duration. During this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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
@@ -373,7 +383,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +426,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +569,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3219,7 +3236,6 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
TimestampTz *delete_time)
{
TransactionId oldestxmin;
- ReplicationSlot *slot;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3229,32 +3245,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
return false;
/*
- * For conflict detection, we use the conflict slot's xmin value instead
- * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
- * a threshold to identify tuples that were recently deleted. These tuples
- * are not visible to concurrent transactions, but we log an
- * update_deleted conflict if such a tuple matches the remote update being
- * applied.
- *
- * Although GetOldestNonRemovableTransactionId() can return a value older
- * than the slot's xmin, for our current purpose it is acceptable to treat
- * tuples deleted by transactions prior to slot.xmin as update_missing
- * conflicts.
+ * For conflict detection, we use the leader worker's
+ * oldest_nonremovable_xid value instead of invoking
+ * GetOldestNonRemovableTransactionId() or relying on the conflict
+ * detection slot's xmin. The oldest_nonremovable_xid acts as a threshold
+ * to identify tuples that were recently deleted. These tuples are not
+ * visible to concurrent transactions, but we log an update_deleted
+ * conflict if such a tuple matches the remote update being applied.
*
- * Ideally, we would use oldest_nonremovable_xid, which is directly
- * maintained by the leader apply worker. However, this value is not
- * available to table synchronization or parallel apply workers, making
- * slot.xmin a practical alternative in those contexts.
+ * Although GetOldestNonRemovableTransactionId() and slot.xmin can return
+ * a value older than the oldest_nonremovable_xid, for our current purpose
+ * it is acceptable to treat tuples deleted by transactions prior to
+ * oldest_nonremovable_xid as update_missing conflicts.
*/
- slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+ if (am_leader_apply_worker())
+ {
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+ }
+ else
+ {
+ LogicalRepWorker *leader;
- Assert(slot);
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
- SpinLockAcquire(&slot->mutex);
- oldestxmin = slot->data.xmin;
- SpinLockRelease(&slot->mutex);
+ SpinLockAcquire(&leader->relmutex);
+ oldestxmin = leader->oldest_nonremovable_xid;
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4143,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4359,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4388,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4509,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4597,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4663,65 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
*/
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,22 +4729,70 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
+
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
+
+ return true;
}
/*
* 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.
+ * 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. However, this maximum interval will not exceed
+ * max_retention_duration.
*
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * The interval is reset to the lesser of 100ms and
+ * max_retention_duration 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.
@@ -4651,6 +4821,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
@@ -5463,6 +5637,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5633,8 +5808,9 @@ DisableSubscriptionAndExit(void)
* an error, as verifying commit timestamps is unnecessary in this
* context.
*/
- if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..bea793456f9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxretention = PQfnumber(res, "submaxretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxretention =
+ atoi(PQgetvalue(res, i, i_submaxretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxretention)
+ appendPQExpBuffer(query, ", max_retention_duration = %d", subinfo->submaxretention);
+
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 dde85ed156c..bcc94ff07cc 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..4aa793d7de7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxretention AS \"%s\"\n",
+ gettext_noop("Max retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..6b20a4404b2 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..55cb9b1eefa 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,15 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +145,13 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxretention; /* The maximum duration (in milliseconds) for
+ * which information useful for conflict
+ * detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..fb4e26a51a4 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,9 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool max_retention_set);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c7f1266fc2f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+ERROR: max_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+NOTICE: max_retention_duration is ineffective when retain_dead_tuples 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 dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..ef0c298d2df 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+
+\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;
--
2.31.1
v66-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v66-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From e2db073ab818d56e9fa587549674603265a3132f Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 26 Aug 2025 18:55:57 +0800
Subject: [PATCH v66 2/3] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized invalidated slot, if at
least one apply worker has confirmed that the retention duration is now within
the max_retention_duration.
---
doc/src/sgml/ref/create_subscription.sgml | 7 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 102 ++++++++---
src/backend/replication/logical/worker.c | 200 ++++++++++++++++++---
4 files changed, 261 insertions(+), 57 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 478b8352164..2d2800a8b9a 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -540,8 +540,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
retention duration exceeded the
<literal>max_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
- disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new subscription with retain_dead_tuples
+ enabled is created.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f9d92702434..5e737f97766 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -855,7 +855,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0211867881..73d2c22ae8b 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -1233,17 +1236,27 @@ ApplyLauncherMain(Datum main_arg)
*/
CreateConflictDetectionSlot();
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * actively retaining dead tuples are enabled. This is
- * required to ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
can_update_xmin &= sub->enabled;
+
+ /*
+ * Initialize slot.xmin as a subscription resumes
+ * retention of information useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+ }
}
if (!sub->enabled)
@@ -1260,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1375,7 +1386,8 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
@@ -1391,7 +1403,27 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_update_xmin)
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1452,23 +1484,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1488,6 +1512,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d378eb08c71..bc552ff036c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,7 +190,7 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
- * If max_retention_duration is defined, one additional phase is
+ * If max_retention_duration is defined, two additional phases are
* involved:
*
* - RDT_STOP_CONFLICT_INFO_RETENTION:
@@ -200,6 +200,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ *
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION at
+ * any phase if the retention has been stopped, but
+ * max_retention_duration is now set to 0.
+ *
* 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
@@ -385,6 +398,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -572,6 +586,10 @@ static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4337,6 +4355,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4359,10 +4384,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4391,6 +4412,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4647,6 +4671,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4674,6 +4710,91 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxretention),
+ errhint("You might need to increase \"%s\".",
+ "max_retention_duration"));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_conflict_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4691,26 +4812,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
@@ -4740,9 +4851,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4773,14 +4883,54 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.31.1
On Wednesday, August 27, 2025 11:30 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
Here is V66 patch set which includes the following changes:
[0001]
* Enhanced documentation according to Dilip's comments [1].
* Simplified logic by directly using the leader's oldest_nonremovable_xid to
locate deleted tuples, according to Amit's comments [2].
* Merged Amit's diff [2].
* Integrated the new NOTICE into the existing CheckSubDeadTupleRetention
function according to Amit's feedback [3].
* Some other adjustments according to feedback from [2] [3].[0002]
* Refactored code according to Shveta's comments [4].
* Removed an unnecessary variable according to Shveta's suggestions [5].
* Some other adjustments according to feedback from [4][5][6].
I noticed that Cfbot failed to compile the document due to a typo after renaming
the subscription option. Here are the updated V67 patches to fix that, only the doc
in 0001 is modified.
Best Regards,
Hou zj
Attachments:
v67-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v67-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From 47f1a769bbb597fd122807a2a89f4f78e1ec44fd Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v67 3/3] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fbd3d1900f4..4679cb70213 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 73d2c22ae8b..6d3559038bd 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1575,7 +1575,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;
@@ -1652,6 +1652,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 6b4a9fb8815..9cfa7c64ced 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.51.0.windows.1
v67-0001-Introduce-a-max_retention_duration-option-to-sub.patchapplication/octet-stream; name=v67-0001-Introduce-a-max_retention_duration-option-to-sub.patchDownload
From 5a707192027e81459f877b228d0d2c470d477915 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v67 1/3] Introduce a 'max_retention_duration' option to
subscriptions.
This commit introduces a subscription option max_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and the apply worker cannot catch up with
the publisher's workload.
If the time spent advancing non-removable transaction ID surpasses the
max_retention_duration threshold, the apply worker would stop retaining
information for conflict detection. The replication slot
pg_conflict_detection.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization once at least one apply worker confirms that the retention
duration is within the max_retention_duration limit.
---
doc/src/sgml/catalogs.sgml | 25 +++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 162 +++++++++----
src/backend/replication/logical/launcher.c | 65 +++---
src/backend/replication/logical/worker.c | 250 ++++++++++++++++++---
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 16 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 5 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 665 insertions(+), 195 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..e9095bedf21 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..12f72ba3167 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..52ac1183e44 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-retention-duration">
+ <term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration for which this subscription's apply worker is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions. The default value is <literal>0</literal>, indicating
+ that the information is retained until it is no longer needed for
+ detection purposes. This value is taken as milliseconds.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
+ <literal>max_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming this replication slot has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions
+ specify a greater value and have not exceeded it, or if they set this
+ option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially resulting in incorrect conflict detection.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..b885890de37 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxretention = subform->submaxretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..fbd3d1900f4 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxretention, subretentionactive,
+ 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 4c01d21b2f3..f9d92702434 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -73,8 +73,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -101,6 +102,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -169,6 +171,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION))
+ opts->maxretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -323,6 +327,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_RETENTION_DURATION;
+ opts->maxretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -580,7 +593,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -647,9 +661,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_retention_duration is appropriate.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ (opts.maxretention > 0));
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -693,6 +711,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1176,6 +1198,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_two_phase = false;
bool check_pub_rdt = false;
bool retain_dead_tuples;
+ int max_retention;
+ bool retention_active;
char *origin;
Subscription *sub;
Form_pg_subscription form;
@@ -1206,6 +1230,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = sub->retaindeadtuples;
origin = sub->origin;
+ max_retention = sub->maxretention;
+ retention_active = sub->retentionactive;
/*
* Don't allow non-superuser modification of a subscription with
@@ -1235,7 +1261,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1401,6 +1429,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only if there's a change in
+ * the retain_dead_tuples option value.
+ *
+ * Automatically marking retention as active when
+ * retain_dead_tuples is enabled may not always be ideal,
+ * especially if retention was previously stopped and the
+ * user toggles retain_dead_tuples without adjusting the
+ * publisher workload. However, this behavior provides a
+ * convenient way for users to manually refresh the
+ * retention status. Since retention will be stopped again
+ * unless the publisher workload is reduced, this approach
+ * is acceptable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1417,13 +1468,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
errhint("Try again after some time.")));
- /*
- * Remind the user that enabling subscription will prevent
- * the accumulation of dead tuples.
- */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
-
/*
* Notify the launcher to manage the replication slot for
* conflict detection. This ensures that replication slot
@@ -1436,6 +1480,26 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ replaces[Anum_pg_subscription_submaxretention - 1] = true;
+
+ max_retention = opts.maxretention;
+ }
+
+ /*
+ * Ensure that the configurations for retain_dead_tuples and
+ * max_retention_duration is appropriate.
+ */
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
+ IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retain_dead_tuples,
+ retention_active,
+ (max_retention > 0));
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1473,9 +1537,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2468,38 +2532,54 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_retention_duration causes no harm,
+ * even when it is ineffective.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool max_retention_set)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
- ereport(ERROR,
- errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
- errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
-
- if (check_guc && !track_commit_timestamp)
- ereport(WARNING,
- errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
- errhint("Consider setting \"%s\" to true.",
- "track_commit_timestamp"));
-
- if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ if (retain_dead_tuples)
+ {
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled && retention_active)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+ }
+ else if (max_retention_set)
+ {
+ ereport(NOTICE,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_dead_tuples") : 0);
+ errmsg("max_retention_duration is ineffective when retain_dead_tuples is disabled"));
+ }
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..c0211867881 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
@@ -102,7 +104,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1181,7 +1184,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool can_advance_xmin = true;
+ bool can_update_xmin = true;
bool retain_dead_tuples = false;
TransactionId xmin = InvalidTransactionId;
@@ -1214,17 +1217,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1232,18 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions
+ * actively retaining dead tuples are enabled. This is
+ * required to ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ if (sub->retentionactive)
+ can_update_xmin &= sub->enabled;
}
if (!sub->enabled)
@@ -1256,7 +1260,9 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,12 +1271,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
- can_advance_xmin = false;
+ if (sub->retaindeadtuples && sub->retentionactive)
+ can_update_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1295,7 +1301,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1327,19 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * However, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to cease retention,
+ * marking it as inactive, the new xmin will be set to
+ * InvalidTransactionId. We then update slot.xmin accordingly to
+ * permit the removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_update_xmin)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1402,17 +1415,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..d378eb08c71 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,6 +190,16 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
+ * If max_retention_duration is defined, one additional phase is
+ * involved:
+ *
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * max_retention_duration. During this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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
@@ -373,7 +383,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +426,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +569,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3219,7 +3236,6 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
TimestampTz *delete_time)
{
TransactionId oldestxmin;
- ReplicationSlot *slot;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3229,32 +3245,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
return false;
/*
- * For conflict detection, we use the conflict slot's xmin value instead
- * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
- * a threshold to identify tuples that were recently deleted. These tuples
- * are not visible to concurrent transactions, but we log an
- * update_deleted conflict if such a tuple matches the remote update being
- * applied.
- *
- * Although GetOldestNonRemovableTransactionId() can return a value older
- * than the slot's xmin, for our current purpose it is acceptable to treat
- * tuples deleted by transactions prior to slot.xmin as update_missing
- * conflicts.
+ * For conflict detection, we use the leader worker's
+ * oldest_nonremovable_xid value instead of invoking
+ * GetOldestNonRemovableTransactionId() or relying on the conflict
+ * detection slot's xmin. The oldest_nonremovable_xid acts as a threshold
+ * to identify tuples that were recently deleted. These tuples are not
+ * visible to concurrent transactions, but we log an update_deleted
+ * conflict if such a tuple matches the remote update being applied.
*
- * Ideally, we would use oldest_nonremovable_xid, which is directly
- * maintained by the leader apply worker. However, this value is not
- * available to table synchronization or parallel apply workers, making
- * slot.xmin a practical alternative in those contexts.
+ * Although GetOldestNonRemovableTransactionId() and slot.xmin can return
+ * a value older than the oldest_nonremovable_xid, for our current purpose
+ * it is acceptable to treat tuples deleted by transactions prior to
+ * oldest_nonremovable_xid as update_missing conflicts.
*/
- slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+ if (am_leader_apply_worker())
+ {
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+ }
+ else
+ {
+ LogicalRepWorker *leader;
- Assert(slot);
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
- SpinLockAcquire(&slot->mutex);
- oldestxmin = slot->data.xmin;
- SpinLockRelease(&slot->mutex);
+ SpinLockAcquire(&leader->relmutex);
+ oldestxmin = leader->oldest_nonremovable_xid;
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4143,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4359,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4388,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4509,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4597,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4663,65 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
*/
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,22 +4729,70 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
+ rdt_data->table_sync_wait_time = 0;
+}
+
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
+
+ return true;
}
/*
* 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.
+ * 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. However, this maximum interval will not exceed
+ * max_retention_duration.
*
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * The interval is reset to the lesser of 100ms and
+ * max_retention_duration 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.
@@ -4651,6 +4821,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
@@ -5463,6 +5637,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5633,8 +5808,9 @@ DisableSubscriptionAndExit(void)
* an error, as verifying commit timestamps is unnecessary in this
* context.
*/
- if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..bea793456f9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxretention = PQfnumber(res, "submaxretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxretention =
+ atoi(PQgetvalue(res, i, i_submaxretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxretention)
+ appendPQExpBuffer(query, ", max_retention_duration = %d", subinfo->submaxretention);
+
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 dde85ed156c..bcc94ff07cc 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..4aa793d7de7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxretention AS \"%s\"\n",
+ gettext_noop("Max retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..6b20a4404b2 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..55cb9b1eefa 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,15 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +145,13 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxretention; /* The maximum duration (in milliseconds) for
+ * which information useful for conflict
+ * detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..fb4e26a51a4 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,9 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool max_retention_set);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c7f1266fc2f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+ERROR: max_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+NOTICE: max_retention_duration is ineffective when retain_dead_tuples 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 dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..ef0c298d2df 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+
+\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;
--
2.51.0.windows.1
v67-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v67-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From c7b3f676c5992ace39d6f3479db6a1d762f94ff2 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 26 Aug 2025 18:55:57 +0800
Subject: [PATCH v67 2/3] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized invalidated slot, if at
least one apply worker has confirmed that the retention duration is now within
the max_retention_duration.
---
doc/src/sgml/ref/create_subscription.sgml | 7 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 102 ++++++++---
src/backend/replication/logical/worker.c | 200 ++++++++++++++++++---
4 files changed, 261 insertions(+), 57 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 52ac1183e44..6244c763d97 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -540,8 +540,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
retention duration exceeded the
<literal>max_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
- disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new subscription with retain_dead_tuples
+ enabled is created.
</para>
<para>
Note that overall retention will not stop if other subscriptions
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f9d92702434..5e737f97766 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -855,7 +855,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0211867881..73d2c22ae8b 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -102,9 +102,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -1233,17 +1236,27 @@ ApplyLauncherMain(Datum main_arg)
*/
CreateConflictDetectionSlot();
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * actively retaining dead tuples are enabled. This is
- * required to ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
can_update_xmin &= sub->enabled;
+
+ /*
+ * Initialize slot.xmin as a subscription resumes
+ * retention of information useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+ }
}
if (!sub->enabled)
@@ -1260,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers that enables retain_dead_tuples.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1375,7 +1386,8 @@ ApplyLauncherMain(Datum main_arg)
* in *xmin.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
@@ -1391,7 +1403,27 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if the
+ * latter is invalid. This ensures the apply worker continues to maintain
+ * the oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ if (!can_update_xmin)
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1452,23 +1484,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1488,6 +1512,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d378eb08c71..bc552ff036c 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -190,7 +190,7 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
- * If max_retention_duration is defined, one additional phase is
+ * If max_retention_duration is defined, two additional phases are
* involved:
*
* - RDT_STOP_CONFLICT_INFO_RETENTION:
@@ -200,6 +200,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is triggered when retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ *
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION at
+ * any phase if the retention has been stopped, but
+ * max_retention_duration is now set to 0.
+ *
* 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
@@ -385,6 +398,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -572,6 +586,10 @@ static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4337,6 +4355,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4359,10 +4384,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4391,6 +4412,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4647,6 +4671,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4674,6 +4710,91 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("The retention duration for information used in conflict detection has exceeded the maximum limit of %u ms.",
+ MySubscription->maxretention),
+ errhint("You might need to increase \"%s\".",
+ "max_retention_duration"));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_conflict_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4691,26 +4812,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
@@ -4740,9 +4851,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4773,14 +4883,54 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid);
+}
+
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
--
2.51.0.windows.1
On Thu, Aug 28, 2025 at 8:02 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I noticed that Cfbot failed to compile the document due to a typo after renaming
the subscription option. Here are the updated V67 patches to fix that, only the doc
in 0001 is modified.
Please find a few comments:
patch001 :
1)
Assert is hit while testing patch001 alone:
TRAP: failed Assert("TransactionIdIsValid(nonremovable_xid)"), File:
"launcher.c", Line: 1394, PID: 55350.
Scenario:
I have 2 subs, both have stopped retention. Now on one of the sub, if I do this:
--switch off rdt:
alter subscription sub1 disable;
alter subscription sub1 set (retain_dead_tuples=off);
alter subscription sub1 enable;
--switch back rdt, launcher asserts after this
alter subscription sub1 disable;
alter subscription sub1 set (retain_dead_tuples=on);
alter subscription sub1 enable;
2)
+ Maximum duration for which this subscription's apply worker
is allowed
+ to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled for the associated
+ subscriptions.
associated subscriptions --> associated subscription (since we are
talking about 'this subscription's apply worker')
3)
+ The information useful for conflict detection is no longer
retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration exceeded the
A trivial thing: 'retention duration has exceeded' sounds better to me.
~~
patch002:
(feel free to defer these comments if we are focusing on patch001 right now):
4)
stop_conflict_info_retention() has the correct message and detail in
patch01 while in patch02, it is switched back to the older one (wrong
one). Perhaps some merge mistake
5)
resume_conflict_info_retention() still refers to the old GUC name:
max_conflict_retention_duration.
6)
In compute_min_nonremovable_xid(), shall we have
Assert(TransactionIdIsValid(nonremovable_xid)) before assigning it to
worker->oldest_nonremovable_xid? Here:
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
7)
Now since compute_min_nonremovable_xid() is also taking care of
assigning valid xid to the worker, shall we update that in comment as
well? We can have one more line:
'Additionally if any of the apply workers has invalid xid, assign
slot's xmin to it.'
thanks
Shveta
On Thu, Aug 28, 2025 at 8:02 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I noticed that Cfbot failed to compile the document due to a typo after renaming
the subscription option. Here are the updated V67 patches to fix that, only the doc
in 0001 is modified.
I have made a number of cosmetic and comment changes in the attached
atop 0001 patch. Kindly include in next version, if you think they are
good to include.
--
With Regards,
Amit Kapila.
Attachments:
v65_amit_1.patch.txttext/plain; charset=US-ASCII; name=v65_amit_1.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 29d0c9a6e45..3df1828e755 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4546,8 +4546,8 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
return;
/*
- * Stop retaining conflict information if required (See
- * should_stop_conflict_info_retention() for details).
+ * We don't need to maintain oldest_nonremovable_xid if we decide
+ * to stop retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
return;
@@ -4650,8 +4650,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
}
/*
- * Stop retaining conflict information if required (See
- * should_stop_conflict_info_retention() for details).
+ * We don't need to maintain oldest_nonremovable_xid if we decide
+ * to stop retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
return;
@@ -4771,16 +4771,16 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
}
/*
- * Check whether conflict information retention should be stopped because the
- * wait time has exceeded the maximum limit (max_conflict_retention_duration).
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_conflict_retention_duration).
*
- * If retention should be stopped, proceed to the
+ * If retention should be stopped, transition to the
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Currently, the retention will not resume automatically unless user manually
- * disables retain_dead_tuples and re-enables it after confirming that the
- * replication slot has been dropped.
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4802,9 +4802,10 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
now = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
/*
- * Return if the wait time has not exceeded the maximum limit
- * (max_conflict_retention_duration). The time spent waiting for table
- * synchronization is not counted, as it's an infrequent operation.
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_conflict_retention_duration). Time spent waiting for table
+ * synchronization is excluded from this calculation, as it occurs
+ * infrequently.
*/
if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
MySubscription->maxconflretention +
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index b86c759394f..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -95,8 +95,8 @@ typedef struct LogicalRepWorker
* named "pg_conflict_detection". It asynchronously collects this ID to
* decide when to advance the xmin value of the slot.
*
- * This ID would be set to InvalidTransactionId if the apply worker has
- * stopped retaining information useful for conflict detection.
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
On Thu, Aug 28, 2025 at 4:39 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Aug 28, 2025 at 8:02 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I noticed that Cfbot failed to compile the document due to a typo after renaming
the subscription option. Here are the updated V67 patches to fix that, only the doc
in 0001 is modified.I have made a number of cosmetic and comment changes in the attached
atop 0001 patch. Kindly include in next version, if you think they are
good to include.
Sorry, by mistake, I attached the wrong file. Please find the correct one now.
--
With Regards,
Amit Kapila.
Attachments:
v67_amit_1.diff.txttext/plain; charset=US-ASCII; name=v67_amit_1.diff.txtDownload
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 52ac1183e44..8cf90eda787 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -526,12 +526,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
<listitem>
<para>
- Maximum duration for which this subscription's apply worker is allowed
- to retain the information useful for conflict detection when
- <literal>retain_dead_tuples</literal> is enabled for the associated
- subscriptions. The default value is <literal>0</literal>, indicating
- that the information is retained until it is no longer needed for
- detection purposes. This value is taken as milliseconds.
+ Maximum duration in milliseconds for which this subscription's apply worker
+ is allowed to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled. The default value
+ is <literal>0</literal>, indicating that the information is retained
+ until it is no longer needed for detection purposes.
</para>
<para>
The information useful for conflict detection is no longer retained if
@@ -541,12 +540,13 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>max_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming this replication slot has been dropped.
+ re-enable it after confirming replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> has been dropped.
</para>
<para>
- Note that overall retention will not stop if other subscriptions
- specify a greater value and have not exceeded it, or if they set this
- option to 0.
+ Note that overall retention will not stop if other subscriptions that
+ have a value greater than 0 for this parameter have not exceeded it,
+ or if they set this option to 0.
</para>
<para>
This option is effective only when
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index fbd3d1900f4..c77fa0234bb 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,7 +1389,7 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, submaxretention, subretentionactive,
+ subretaindeadtuples, submaxretention, subretentionactive,
subslotname, subsynccommit, subpublications, suborigin)
ON pg_subscription TO public;
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index f9d92702434..aeb0c413067 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -662,8 +662,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
}
/*
- * Ensure that the configurations for retain_dead_tuples and
- * max_retention_duration is appropriate.
+ * Ensure that system configuration paramters are set appropriately to
+ * support retain_dead_tuples and max_retention_duration.
*/
CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
opts.retaindeadtuples, opts.retaindeadtuples,
@@ -1490,8 +1490,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
}
/*
- * Ensure that the configurations for retain_dead_tuples and
- * max_retention_duration is appropriate.
+ * Ensure that system configuration paramters are set appropriately to
+ * support retain_dead_tuples and max_retention_duration.
*/
if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index c0211867881..969d48a1d5d 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -23,7 +23,6 @@
#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"
@@ -1258,7 +1257,7 @@ ApplyLauncherMain(Datum main_arg)
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers that enables retain_dead_tuples.
+ * workers.
*/
if (sub->retaindeadtuples &&
sub->retentionactive &&
@@ -1328,11 +1327,10 @@ ApplyLauncherMain(Datum main_arg)
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
*
- * However, if all apply workers for subscriptions with
- * retain_dead_tuples enabled have requested to cease retention,
- * marking it as inactive, the new xmin will be set to
- * InvalidTransactionId. We then update slot.xmin accordingly to
- * permit the removal of dead tuples.
+ * Additionally, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to stop retention,
+ * the slot's xmin will be set to InvalidTransactionId allowing the
+ * removal of dead tuples.
*/
if (MyReplicationSlot)
{
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d378eb08c71..df16cbe3716 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,6 +173,14 @@
* Advance the non-removable transaction ID if the current flush location has
* reached or surpassed the last received WAL position.
*
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * configured max_retention_duration. In this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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) ->
@@ -190,16 +198,6 @@
* update_deleted is necessary, as the UPDATEs in remote transactions should be
* ignored if their timestamp is earlier than that of the dead tuples.
*
- * If max_retention_duration is defined, one additional phase is
- * involved:
- *
- * - RDT_STOP_CONFLICT_INFO_RETENTION:
- * This phase is triggered when the wait time in either the
- * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
- * max_retention_duration. During this phase,
- * pg_subscription.subretentionactive is updated to false within a new
- * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
- *
* 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
@@ -569,9 +567,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData* rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
-static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3247,16 +3245,16 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
/*
* For conflict detection, we use the leader worker's
* oldest_nonremovable_xid value instead of invoking
- * GetOldestNonRemovableTransactionId() or relying on the conflict
- * detection slot's xmin. The oldest_nonremovable_xid acts as a threshold
- * to identify tuples that were recently deleted. These tuples are not
- * visible to concurrent transactions, but we log an update_deleted
- * conflict if such a tuple matches the remote update being applied.
+ * GetOldestNonRemovableTransactionId() or using the conflict detection
+ * slot's xmin. The oldest_nonremovable_xid acts as a threshold to identify
+ * tuples that were recently deleted. These deleted tuples are no longer
+ * visible to concurrent transactions. However, if a remote update matches
+ * such a tuple, we log an update_deleted conflict.
*
- * Although GetOldestNonRemovableTransactionId() and slot.xmin can return
- * a value older than the oldest_nonremovable_xid, for our current purpose
- * it is acceptable to treat tuples deleted by transactions prior to
- * oldest_nonremovable_xid as update_missing conflicts.
+ * While GetOldestNonRemovableTransactionId() and slot.xmin may return
+ * transaction IDs older than oldest_nonremovable_xid, for our current
+ * purpose, it is acceptable to treat tuples deleted by transactions prior
+ * to oldest_nonremovable_xid as update_missing conflicts.
*/
if (am_leader_apply_worker())
{
@@ -4669,6 +4667,55 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
process_rdt_phase_transition(rdt_data, false);
}
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData* rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
+ */
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return true;
+}
+
/*
* Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
*/
@@ -4732,67 +4779,22 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time = 0;
}
-/*
- * Check whether conflict information retention should be stopped due to
- * exceeding the maximum wait time (max_retention_duration).
- *
- * If retention should be stopped, transition to the
- * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
- * false.
- *
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
- */
-static bool
-should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
-{
- TimestampTz now;
-
- Assert(TransactionIdIsValid(rdt_data->candidate_xid));
- Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
- rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
-
- if (!MySubscription->maxretention)
- 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
-
- /*
- * Return early if the wait time has not exceeded the configured maximum
- * (max_retention_duration). Time spent waiting for table synchronization
- * is excluded from this calculation, as it occurs infrequently.
- */
- if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
- MySubscription->maxretention +
- rdt_data->table_sync_wait_time))
- return false;
-
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
-
- return true;
-}
-
/*
* 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. However, this maximum interval will not exceed
- * max_retention_duration.
+ * If there is no activity on the node, we progressively double the interval
+ * used to advance non-removable transaction ID. This helps conserve CPU
+ * and network resources when there's little benefit to frequent updates.
+ *
+ * The interval is capped by the lowest of the following:
+ * - wal_receiver_status_interval (if set),
+ * - a default maximum of 3 minutes,
+ * - max_retention_duration.
*
- * The interval is reset to the lesser of 100ms and
- * max_retention_duration once there is some activity on the node.
+ * This ensures the interval never exceeds the retention boundary, even if
+ * other limits are higher. Once activity resumes on the node, the interval
+ * is reset to lesser of 100ms and max_retention_duration, allowing timely
+ * advancement of non-removable transaction ID.
*
* 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.
Hi,
As per the test results in [1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com, the TPS drop observed on the
subscriber with update_deleted enabled was mainly because only a
single apply worker was handling the replication workload from
multiple concurrent publisher clients.
The following performance benchmarks were conducted to evaluate the
improvements using parallel apply when update_deleted
(retain_dead_tuples) is enabled, under heavy workloads, without
leveraging row filters or multiple subscriptions to distribute the
load.
Note: The earlier tests from[1]/messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com are repeated with few workload
modifications to see the improvements using parallel-apply.
Highlights
===============
- No regression was observed when running pgbench individually on
either Pub or Sub nodes.
- When pgbench was run on both Pub and Sub, performance improved
significantly with the parallel apply patch. With just 4 workers, Sub
was able to catch up with Pub without regression.
- With max_conflict_retention_duration=60s, retention on Sub was not
stopped when using 4 or more parallel workers.
Machine details
===============
Intel(R) Xeon(R) CPU E7-4890 v2 @ 2.80GHz CPU(s) :88 cores, - 503 GiB RAM
Source code:
===============
- pgHead(e9a31c0cc60) and parallel-apply v1 patch[2]/messages/by-id/OS0PR01MB5716D43CB68DB8FFE73BF65D942AA@OS0PR01MB5716.jpnprd01.prod.outlook.com
- additionally used v64-0001 of update_deleted for
max_conflict_retention_duration related tests.
Test-01: pgbench on publisher
============================
Setup:
---------
Pub --> Sub
- Two nodes created in pub-sub logical replication setup.
- Both nodes have the same set of pgbench tables created with scale=60.
- The Sub node is subscribed to all the changes from the Pub's
pgbench tables and the subscription has retain_dead_tuples = on
Workload:
--------------
- Run default pgbench(read-write) only on Pub with #clients=40 and
run duration=10 minutes
Results:
------------
#run pgHead_TPS pgHead+v1_patch_TPS
1 41135.71241 39922.7163
2 40466.23865 39980.29034
3 40578.16374 39867.44929
median 40578.16374 39922.7163
- No regression.
~~~
Test-02: pgbench on subscriber
========================
Setup: same as Test-01
Workload:
--------------
- Run default pgbench(read-write) only on Sub node with #clients=40
and run duration=10 minutes
Results:
-----------
#run pgHead_TPS pgHead+v1_patch_TPS
1 42173.90504 42071.18058
2 41836.10027 41839.84837
3 41921.81233 41494.9918
median 41921.81233 41839.84837
- No regression.
~~~
Test-03: pgbench on both sides
========================
Setup:
------
Pub --> Sub
- Two nodes created in a pub-sub logical replication setup.
- Both nodes have different sets of pgbench tables created with scale=60.
- The sub node also has Pub's pgbench tables and is subscribed to all
the changes.
Workload:
--------------
- Run default pgbench(read-write) on both Pub and Sub nodes for their
respective pgbench tables
- Both pgbench runs are with #clients=15 and duration=10 minutes
Observations:
--------------
- On pgHead when retain_dead_tuples=ON, the Sub's TPS reduced by ~76%
- With the parallel apply patch, performance improves significantly
as parallel workers increase, since conflict_slot.xmin advances more
quickly.
- With just 4 parallel workers, subscription TPS matches the baseline
(no regression).
- Performance remains consistent at 8 and 16 workers.
Detailed results:
------------------
case-1:
- The base case pgHead(e9a31c0cc60) and retain_dead_tuples=OFF
#run Pub_tps Sub_tps
1 17140.08647 16994.63269
2 17421.28513 17445.62828
3 17167.57801 17070.86979
median 17167.57801 17070.86979
case-2:
- pgHead(e9a31c0cc60) and retain_dead_tuples=ON
#run Pub_tps Sub_tps
1 18667.29343 4135.884924
2 18200.90297 4178.713784
3 18309.87093 4227.330234
median 18309.87093 4178.713784
- The Sub sees a ~76% of TPS reduction by default on head.
case-3:
- pgHead(e9a31c0cc60)+ v1_parallel_apply_patch and retain_dead_tuples=ON
- number of parallel apply workers varied as 2,4,8,16
3a) #workers=2
#run Pub_tps Sub_tps
1 18336.98565 4244.072357
2 18629.96658 4231.887288
3 18152.92036 4253.293648
median 18336.98565 4244.072357
- There is no significant TPS improvement with 2 parallel workers,
~76% TPS reduction
3b) #workers=4
#run Pub_tps Sub_tps
1 16796.49468 16850.05362
2 16834.06057 16757.73115
3 16647.78486 16762.9107
median 16796.49468 16762.9107
- No regression
3c) #workers=8
#run Pub_tps Sub_tps
1 17105.38794 16778.38209
2 16783.5085 16780.20492
3 16806.97569 16642.87521
median 16806.97569 16778.38209
- No regression
3d) #workers=16
#run Pub_tps Sub_tps
1 16827.20615 16770.92293
2 16860.10188 16745.2968
3 16808.2148 16668.47974
median 16827.20615 16745.2968
- No regression.
~~~
Test-04. pgbench on both side, and max_conflict_retention_duration was tuned
========================================================================
Setup:
-------
Pub --> Sub
- setup is same as Test-03(above)
- Additionally, subscription option max_conflict_retention_duration=60s
Workload:
-------------
- Run default pgbench(read-write) on both Pub and Sub nodes for their
respective pgbench tables
- Started with 15 clients on both sides.
- When conflict_slot.xmin becomes NULL on Sub, pgbench was paused to
let the subscription catch up. Then reduced publisher clients by half
and resumed pgbench. Here, slot.xmin becomes NULL to indicate conflict
retention is stopped under high publisher load but stays non-NULL when
Sub is able to catchup with Pub's load.
- Total duration of pgbench run is 10 minutes (600s).
Observations:
------------------
- Without the parallel apply patch, publisher clients reduced from
15->7->3,and finally the retention was not stopped at 3 clients and
slot.xmin remained non-NULL.
- With the parallel apply patch, using 2 workers the subscription
handled up to 7 publisher clients without stopping the conflict
retention.
- With 4+ workers, retention continued for the full 10 minutes and
Sub TPS showed no regression.
Detailed results:
-----------------
case-1:
- pgHead(e9a31c0cc60) + v64-001 and retain_dead_tuples=ON
On publisher:
#cleints durations[s] TPS
15 73 17953.52
7 100 9141.9
3 426 4286.381132
On Subscriber:
#cleints durations[s] TPS
15 73 10626.67
15 99 10271.35
15 431 19467.07612
~~~
case-2:
- pgHead(e9a31c0cc60) + v64-001 + v1_parallel-apply patch[2]/messages/by-id/OS0PR01MB5716D43CB68DB8FFE73BF65D942AA@OS0PR01MB5716.jpnprd01.prod.outlook.com and
retain_dead_tuples=ON
- number of parallel apply workers varied as 2,4,8
2a) #workers=2
On publisher:
#cleints durations[s] TPS
15 87 17318.3
7 512 9063.506025
On Subscriber:
#cleints durations[s] TPS
15 87 10299.66
15 512 18426.44818
2b) #workers=4
On publisher:
#cleints durations[s] TPS
15 600 16953.40302
On Subscriber:
#cleints durations[s] TPS
15 600 16812.15289
2c) #workers=8
On publisher:
#cleints durations[s] TPS
15 600 16946.91636
On Subscriber:
#cleints durations[s] TPS
15 600 16708.12774
~~~~
The scripts used for all the tests are attached.
[1]: /messages/by-id/OSCPR01MB1496663AED8EEC566074DFBC9F54CA@OSCPR01MB14966.jpnprd01.prod.outlook.com
[2]: /messages/by-id/OS0PR01MB5716D43CB68DB8FFE73BF65D942AA@OS0PR01MB5716.jpnprd01.prod.outlook.com
--
Thanks,
Nisha
Attachments:
On Thursday, August 28, 2025 7:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Aug 28, 2025 at 8:02 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I noticed that Cfbot failed to compile the document due to a typo
after renaming the subscription option. Here are the updated V67
patches to fix that, only the doc in 0001 is modified.I have made a number of cosmetic and comment changes in the attached atop
0001 patch. Kindly include in next version, if you think they are good to include.
Thanks ! The changes look good to me, I have merged them into V68 patch set.
Here is the new version patch set which also addressed Shveta's comments[1]/messages/by-id/CAJpy0uD=MLy77JZ78_J4H3XCV1mCA=iUPHuFC5Vt4EKyj6zfjg@mail.gmail.com.
[1]: /messages/by-id/CAJpy0uD=MLy77JZ78_J4H3XCV1mCA=iUPHuFC5Vt4EKyj6zfjg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v68-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v68-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From f8d9954722b0bad4b02eb92a9be80c9ea2bdc473 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v68 3/3] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c77fa0234bb..b47109ab11d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 411259cc23a..176f9e8ea41 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1575,7 +1575,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;
@@ -1652,6 +1652,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 6b4a9fb8815..9cfa7c64ced 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.51.0.windows.1
v68-0001-Introduce-a-max_retention_duration-option-to-sub.patchapplication/octet-stream; name=v68-0001-Introduce-a-max_retention_duration-option-to-sub.patchDownload
From 62d739af6cb6752c3f2b524d4878e224efdf7f2b Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v68 1/2] Introduce a 'max_retention_duration' option to
subscriptions.
This commit introduces a subscription option max_retention_duration,
designed to prevent excessive accumulation of dead tuples when subscription with
retain_dead_tuples enabled is present and the apply worker cannot catch up with
the publisher's workload.
If the time spent advancing non-removable transaction ID surpasses the
max_retention_duration threshold, the apply worker would stop retaining
information for conflict detection. The replication slot
pg_conflict_detection.xmin will be set to InvalidTransactionId if all apply
workers associated with the subscription, where retain_dead_tuples is enabled,
confirm that the retention duration exceeded the
max_retention_duration.
Additionally, retention status is recorded in the pg_subscription catalog
(subretentionactive) to prevent unnecessary retention initiation upon server
restarts.
In this patch, a replication slot will not be automatically re-initialized.
Users can disable retain_dead_tuples and re-enable it manually to resume the
retention. An upcoming patch will include support for automatic slot
re-initialization once at least one apply worker confirms that the retention
duration is within the max_retention_duration limit.
---
doc/src/sgml/catalogs.sgml | 25 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 163 +++++++++----
src/backend/replication/logical/launcher.c | 76 +++---
src/backend/replication/logical/worker.c | 258 +++++++++++++++++----
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 16 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 5 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 680 insertions(+), 200 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..e9095bedf21 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..12f72ba3167 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..0b82e1e3b71 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-retention-duration">
+ <term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration in milliseconds for which this subscription's apply worker
+ is allowed to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled. The default value
+ is <literal>0</literal>, indicating that the information is retained
+ until it is no longer needed for detection purposes.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration has exceeded the
+ <literal>max_retention_duration</literal> set within the
+ corresponding subscription. To re-enable retention manually, you can
+ disable <literal>retain_dead_tuples</literal> for all subscriptions and
+ re-enable it after confirming replication slot
+ <quote><literal>pg_conflict_detection</literal></quote> has been dropped.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions that
+ have a value greater than 0 for this parameter have not exceeded it,
+ or if they set this option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially resulting in incorrect conflict detection.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..b885890de37 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxretention = subform->submaxretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..c77fa0234bb 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxretention, subretentionactive,
+ 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 0d74398faf3..82cf65fae73 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -72,8 +72,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -100,6 +101,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -168,6 +170,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION))
+ opts->maxretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -322,6 +326,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_RETENTION_DURATION;
+ opts->maxretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -579,7 +592,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -646,9 +660,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that system configuration paramters are set appropriately to
+ * support retain_dead_tuples and max_retention_duration.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ (opts.maxretention > 0));
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -692,6 +710,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1197,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_two_phase = false;
bool check_pub_rdt = false;
bool retain_dead_tuples;
+ int max_retention;
+ bool retention_active;
char *origin;
Subscription *sub;
Form_pg_subscription form;
@@ -1205,6 +1229,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = sub->retaindeadtuples;
origin = sub->origin;
+ max_retention = sub->maxretention;
+ retention_active = sub->retentionactive;
/*
* Don't allow non-superuser modification of a subscription with
@@ -1234,7 +1260,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1400,6 +1428,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only if there's a change in
+ * the retain_dead_tuples option value.
+ *
+ * Automatically marking retention as active when
+ * retain_dead_tuples is enabled may not always be ideal,
+ * especially if retention was previously stopped and the
+ * user toggles retain_dead_tuples without adjusting the
+ * publisher workload. However, this behavior provides a
+ * convenient way for users to manually refresh the
+ * retention status. Since retention will be stopped again
+ * unless the publisher workload is reduced, this approach
+ * is acceptable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1416,13 +1467,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
errhint("Try again after some time.")));
- /*
- * Remind the user that enabling subscription will prevent
- * the accumulation of dead tuples.
- */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
-
/*
* Notify the launcher to manage the replication slot for
* conflict detection. This ensures that replication slot
@@ -1435,6 +1479,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ replaces[Anum_pg_subscription_submaxretention - 1] = true;
+
+ max_retention = opts.maxretention;
+ }
+
+ /*
+ * Ensure that system configuration paramters are set
+ * appropriately to support retain_dead_tuples and
+ * max_retention_duration.
+ */
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
+ IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retain_dead_tuples,
+ retention_active,
+ (max_retention > 0));
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1472,9 +1537,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2467,38 +2532,54 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_retention_duration causes no harm,
+ * even when it is ineffective.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool max_retention_set)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
- ereport(ERROR,
- errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
- errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
-
- if (check_guc && !track_commit_timestamp)
- ereport(WARNING,
- errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
- errhint("Consider setting \"%s\" to true.",
- "track_commit_timestamp"));
-
- if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ if (retain_dead_tuples)
+ {
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled && retention_active)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+ }
+ else if (max_retention_set)
+ {
+ ereport(NOTICE,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_dead_tuples") : 0);
+ errmsg("max_retention_duration is ineffective when retain_dead_tuples is disabled"));
+ }
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..657f4f25a05 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -43,6 +43,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
@@ -102,7 +103,7 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
/*
@@ -152,6 +153,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1181,7 +1183,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool can_advance_xmin = true;
+ bool can_update_xmin = true;
bool retain_dead_tuples = false;
TransactionId xmin = InvalidTransactionId;
@@ -1214,17 +1216,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1231,18 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ /*
+ * Can't advance xmin of the slot unless all the subscriptions
+ * actively retaining dead tuples are enabled. This is
+ * required to ensure that we don't advance the xmin of
+ * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
+ * enabled. Otherwise, we won't be able to detect conflicts
+ * reliably for such a subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ if (sub->retentionactive)
+ can_update_xmin &= sub->enabled;
}
if (!sub->enabled)
@@ -1254,9 +1257,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers that enables retain_dead_tuples.
+ * workers.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (TransactionIdIsValid(MyReplicationSlot->data.xmin) &&
+ sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,12 +1271,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
- can_advance_xmin = false;
+ if (sub->retaindeadtuples && sub->retentionactive)
+ can_update_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1295,7 +1301,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1327,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * Additionally, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to stop retention, the
+ * slot's xmin will be set to InvalidTransactionId allowing the
+ * removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_update_xmin)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1378,7 +1390,15 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Return if the apply worker has stopped retention concurrently.
+ *
+ * Although this function is invoked only when retentionactive is true,
+ * the apply worker might stop retention after the launcher fetches the
+ * retentionactive flag.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1422,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..d75ed4aaf97 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,6 +173,14 @@
* Advance the non-removable transaction ID if the current flush location has
* reached or surpassed the last received WAL position.
*
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * configured max_retention_duration. In this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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) ->
@@ -373,7 +381,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -415,6 +424,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +567,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3219,7 +3234,6 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
TimestampTz *delete_time)
{
TransactionId oldestxmin;
- ReplicationSlot *slot;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3229,32 +3243,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
return false;
/*
- * For conflict detection, we use the conflict slot's xmin value instead
- * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
- * a threshold to identify tuples that were recently deleted. These tuples
- * are not visible to concurrent transactions, but we log an
- * update_deleted conflict if such a tuple matches the remote update being
- * applied.
+ * For conflict detection, we use the leader worker's
+ * oldest_nonremovable_xid value instead of invoking
+ * GetOldestNonRemovableTransactionId() or using the conflict detection
+ * slot's xmin. The oldest_nonremovable_xid acts as a threshold to
+ * identify tuples that were recently deleted. These deleted tuples are no
+ * longer visible to concurrent transactions. However, if a remote update
+ * matches such a tuple, we log an update_deleted conflict.
*
- * Although GetOldestNonRemovableTransactionId() can return a value older
- * than the slot's xmin, for our current purpose it is acceptable to treat
- * tuples deleted by transactions prior to slot.xmin as update_missing
- * conflicts.
- *
- * Ideally, we would use oldest_nonremovable_xid, which is directly
- * maintained by the leader apply worker. However, this value is not
- * available to table synchronization or parallel apply workers, making
- * slot.xmin a practical alternative in those contexts.
+ * While GetOldestNonRemovableTransactionId() and slot.xmin may return
+ * transaction IDs older than oldest_nonremovable_xid, for our current
+ * purpose, it is acceptable to treat tuples deleted by transactions prior
+ * to oldest_nonremovable_xid as update_missing conflicts.
*/
- slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+ if (am_leader_apply_worker())
+ {
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+ }
+ else
+ {
+ LogicalRepWorker *leader;
- Assert(slot);
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
- SpinLockAcquire(&slot->mutex);
- oldestxmin = slot->data.xmin;
- SpinLockRelease(&slot->mutex);
+ SpinLockAcquire(&leader->relmutex);
+ oldestxmin = leader->oldest_nonremovable_xid;
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4141,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid) &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4357,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
+ return false;
+
return true;
}
@@ -4350,6 +4386,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4507,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4595,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4661,114 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
*/
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return true;
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,22 +4776,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ rdt_data->table_sync_wait_time = 0;
}
/*
* 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.
+ * If there is no activity on the node, we progressively double the interval
+ * used to advance non-removable transaction ID. This helps conserve CPU
+ * and network resources when there's little benefit to frequent updates.
+ *
+ * The interval is capped by the lowest of the following:
+ * - wal_receiver_status_interval (if set),
+ * - a default maximum of 3 minutes,
+ * - max_retention_duration.
*
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * This ensures the interval never exceeds the retention boundary, even if
+ * other limits are higher. Once activity resumes on the node, the interval
+ * is reset to lesser of 100ms and max_retention_duration, allowing timely
+ * advancement of non-removable transaction ID.
*
* 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.
@@ -4651,6 +4823,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
@@ -5463,6 +5639,7 @@ InitializeLogRepWorker(void)
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5633,8 +5810,9 @@ DisableSubscriptionAndExit(void)
* an error, as verifying commit timestamps is unnecessary in this
* context.
*/
- if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..bea793456f9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxretention = PQfnumber(res, "submaxretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxretention =
+ atoi(PQgetvalue(res, i, i_submaxretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxretention)
+ appendPQExpBuffer(query, ", max_retention_duration = %d", subinfo->submaxretention);
+
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 dde85ed156c..bcc94ff07cc 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..4aa793d7de7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxretention AS \"%s\"\n",
+ gettext_noop("Max retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..6b20a4404b2 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..55cb9b1eefa 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,15 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +145,13 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxretention; /* The maximum duration (in milliseconds) for
+ * which information useful for conflict
+ * detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..fb4e26a51a4 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,9 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool max_retention_set);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c7f1266fc2f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+ERROR: max_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+NOTICE: max_retention_duration is ineffective when retain_dead_tuples 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 dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..ef0c298d2df 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+
+\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;
--
2.51.0.windows.1
v68-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v68-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 72a93a16ec69561571a0b023c89897a179925d6d Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Fri, 29 Aug 2025 13:51:33 +0800
Subject: [PATCH v68 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized invalidated slot, if at
least one apply worker has confirmed that the retention duration is now within
the max_retention_duration.
---
doc/src/sgml/ref/create_subscription.sgml | 8 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 116 ++++++++----
src/backend/replication/logical/worker.c | 204 ++++++++++++++++++---
src/include/replication/worker_internal.h | 6 +
5 files changed, 284 insertions(+), 59 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 0b82e1e3b71..0713a2b20b0 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -539,9 +539,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the
corresponding subscription. To re-enable retention manually, you can
- disable <literal>retain_dead_tuples</literal> for all subscriptions and
- re-enable it after confirming replication slot
- <quote><literal>pg_conflict_detection</literal></quote> has been dropped.
+ disable <literal>retain_dead_tuples</literal> and re-enable it.
+ Alternatively, the retention will be automatically resumed once at
+ least one apply worker confirms that the retention duration is within
+ the specified limit, or if a new subscription with retain_dead_tuples
+ enabled is created.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 82cf65fae73..cc622fce58f 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -854,7 +854,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 657f4f25a05..335c6d8ab81 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,9 +101,12 @@ 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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -467,6 +470,7 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->wait_for_initial_xid = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1232,17 +1236,27 @@ ApplyLauncherMain(Datum main_arg)
*/
CreateConflictDetectionSlot();
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * actively retaining dead tuples are enabled. This is
- * required to ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
can_update_xmin &= sub->enabled;
+
+ /*
+ * Initialize slot.xmin as a subscription resumes
+ * retention of information useful for conflict detection.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+ }
}
if (!sub->enabled)
@@ -1259,11 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers.
*/
- if (TransactionIdIsValid(MyReplicationSlot->data.xmin) &&
- sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1372,11 +1383,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * Additionally, if an apply worker has an invalid XID and is requesting to
+ * resume retention, assign the slot's xmin value to it.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool wait_for_xid;
Assert(worker != NULL);
@@ -1388,16 +1404,42 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ wait_for_xid = worker->wait_for_initial_xid;
SpinLockRelease(&worker->relmutex);
/*
- * Return if the apply worker has stopped retention concurrently.
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if
+ * requested. This ensures the apply worker continues to maintain the
+ * oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (wait_for_xid)
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return if the apply worker has stopped retention concurrently and has not
+ * yet resumed.
*
* Although this function is invoked only when retentionactive is true,
* the apply worker might stop retention after the launcher fetches the
* retentionactive flag.
*/
- if (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ if (!can_update_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
@@ -1459,23 +1501,15 @@ update_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1495,6 +1529,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index d75ed4aaf97..43c01b1792d 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION
+ * at any phase if the retention has been stopped, but max_retention_duration
+ * is now set to 0.
+ *
* 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) ->
@@ -383,6 +396,7 @@ typedef enum
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -569,6 +583,10 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4335,6 +4353,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4357,10 +4382,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
- return false;
-
return true;
}
@@ -4389,6 +4410,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4645,6 +4669,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4675,9 +4711,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4708,10 +4743,16 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
@@ -4721,6 +4762,131 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !MySubscription->retentionactive;
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = false;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4738,26 +4904,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 62ea1a00580..58f2dcc7f0b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -100,6 +100,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker is resuming retention and is waiting
+ * for the launcher to initialize oldest_nonremovable_xid.
+ */
+ bool wait_for_initial_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.51.0.windows.1
On Thursday, August 28, 2025 6:07 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Aug 28, 2025 at 8:02 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:I noticed that Cfbot failed to compile the document due to a typo
after renaming the subscription option. Here are the updated V67
patches to fix that, only the doc in 0001 is modified.Please find a few comments:
Thanks for the comments.
patch001 :
1)
Assert is hit while testing patch001 alone:
TRAP: failed Assert("TransactionIdIsValid(nonremovable_xid)"), File:
"launcher.c", Line: 1394, PID: 55350.Scenario:
I have 2 subs, both have stopped retention. Now on one of the sub, if I do this:--switch off rdt:
alter subscription sub1 disable;
alter subscription sub1 set (retain_dead_tuples=off); alter subscription sub1
enable;--switch back rdt, launcher asserts after this alter subscription sub1 disable;
alter subscription sub1 set (retain_dead_tuples=on); alter subscription sub1
enable;
I fixed it by skipping collecting the xid from workers when the overall
retention has been stopped.
When fixing the reported Assert failure, I noticed another race condition that
can hit this Assert: If the launcher has obtained
pg_subscription.retentionactive and has not yet called
compute_min_nonremovable_xid(), then if the apply worker stops retention and
sets oldest_nonremovable_xid to invalid, the launcher will hit the assertion
again when calling compute_min_nonremovable_xid(). I fixed it by adding a check
in compute function:
/*
* Return if the apply worker has stopped retention concurrently.
*
* Although this function is invoked only when retentionactive is true, the
* apply worker might stop retention after the launcher fetches the
* retentionactive flag.
*/
if (!TransactionIdIsValid(nonremovable_xid))
return;
The logic in 0002 is also adjusted to avoid assigning the slot.xmin to worker in this
case.
2) + Maximum duration for which this subscription's apply worker is allowed + to retain the information useful for conflict detection when + <literal>retain_dead_tuples</literal> is enabled for the associated + subscriptions.associated subscriptions --> associated subscription (since we are talking
about 'this subscription's apply worker')
I have reworded this part based on Amit's suggestion.
3) + The information useful for conflict detection is no longer retained if + all apply workers associated with the subscriptions, where + <literal>retain_dead_tuples</literal> is enabled, confirm that the + retention duration exceeded theA trivial thing: 'retention duration has exceeded' sounds better to me.
Changed
~~
patch002:
(feel free to defer these comments if we are focusing on patch001 right now):4)
stop_conflict_info_retention() has the correct message and detail in
patch01 while in patch02, it is switched back to the older one (wrong one).
Perhaps some merge mistake
Right, It a miss when rebasing the 0002. Fixed.
5)
resume_conflict_info_retention() still refers to the old GUC name:
max_conflict_retention_duration.
Fixed.
6)
In compute_min_nonremovable_xid(), shall we have
Assert(TransactionIdIsValid(nonremovable_xid)) before assigning it to
worker->oldest_nonremovable_xid? Here:+ nonremovable_xid = MyReplicationSlot->data.xmin; + + SpinLockAcquire(&worker->relmutex); + worker->oldest_nonremovable_xid = nonremovable_xid; + SpinLockRelease(&worker->relmutex);
Added.
7)
Now since compute_min_nonremovable_xid() is also taking care of assigning
valid xid to the worker, shall we update that in comment as well? We can have
one more line:
'Additionally if any of the apply workers has invalid xid, assign slot's xmin to it.'
Added.
Best Regards,
Hou zj
On Fri, Aug 29, 2025 at 11:49 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the new version patch set which also addressed Shveta's comments[1].
Thanks for the patch.
On 001 alone, I’m observing a behavior where, if sub1 has stopped
retention, and I then create a new subscription sub2, the worker for
sub2 fails to start successfully. It repeatedly starts and exits,
logging the following message:
LOG: logical replication worker for subscription "sub2" will restart
because the option retain_dead_tuples was enabled during startup
Same things happen when I disable and re-enable 'retain_dead_tuple' of
any sub once the slot has invalid xmin.
thanks
Shveta
On Fri, Aug 29, 2025 at 11:49 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the new version patch set which also addressed Shveta's comments[1].
Thanks for the patches here, I tested the v68-001 patch alone, please
find review comments -
1) If a sub is created with retain_dead_tuples=on but disabled, e.g.
postgres=# create subscription sub3 CONNECTION 'dbname=postgres
host=localhost port=8841' PUBLICATION pub3 WITH
(enabled=false,retain_dead_tuples=on);
WARNING: deleted rows to detect conflicts would not be removed until
the subscription is enabled
HINT: Consider setting retain_dead_tuples to false.
NOTICE: created replication slot "sub3" on publisher
CREATE SUBSCRIPTION
In this case, the conflict slot is not created until the sub is
enabled. Also, if the slot already exists but all other subscriptions
have stopped retaining (slot.xmin=NULL), the dead tuple retention will
not start until the slot is recreated.
To me, the above warning seems misleading in this case.
2) A similar situation can happen with ALTER SUBSCRIPTION. For
example, consider two subscriptions where retention has stopped for
both and slot.xmin is NULL:
subname | subenabled | subretaindeadtuples | submaxretention |
subretentionactive
---------+------------+---------------------+-----------------+--------------------
sub2 | t | t | 100 | f
sub1 | t | t | 100 | f
postgres=# select slot_name,active,xmin from pg_replication_slots ;
slot_name | active | xmin
-----------------------+--------+------
pg_conflict_detection | t |
If we try to resume retention only for sub1 by toggling retain_dead_tuples:
postgres=# alter subscription sub1 set (retain_dead_tuples =off);
NOTICE: max_retention_duration is ineffective when retain_dead_tuples
is disabled
ALTER SUBSCRIPTION
postgres=# alter subscription sub1 set (retain_dead_tuples =on);
NOTICE: deleted rows to detect conflicts would not be removed until
the subscription is enabled
ALTER SUBSCRIPTION
2a) Here also the retention NOTICE is ambiguous as slot.xmin remains
NULL. Though, the above steps don't strictly follow the docs (i.e.
slot should be recreated to resume the retention), still the notice
can be confusing for users.
2b) Also, the retention is not resumed for sub1(expected), but still
the subretentionactive is changed to true.
subname | subenabled | subretaindeadtuples | submaxretention |
subretentionactive
---------+------------+---------------------+-----------------+--------------------
sub1 | f | t | 100 | t
sub2 | t | t | 100 | f
I think we should avoid changing subretentionactive to true in such
cases until the slot is recreated and retention is actually resumed.
Thoughts?
--
Thanks,
Nisha
On Sat, Aug 30, 2025 at 10:17 AM Nisha Moond <nisha.moond412@gmail.com> wrote:
On Fri, Aug 29, 2025 at 11:49 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Here is the new version patch set which also addressed Shveta's comments[1].
Thanks for the patches here, I tested the v68-001 patch alone, please
find review comments -
Further review comments for v68-0001 patch -
3) v68 seems to have introduced a bug:
@@ -1254,9 +1257,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers that enables retain_dead_tuples.
+ * workers.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (TransactionIdIsValid(MyReplicationSlot->data.xmin) &&
+ sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
The new check "TransactionIdIsValid(MyReplicationSlot->data.xmin)" can
cause a segmentation fault in the launcher when a default subscription
is created (i.e., retain_dead_tuples=off) and the conflict slot does
not exist.
Perhaps it should first check "sub->retaindeadtuples" before accessing the slot.
--
Thanks,
Nisha
On Thu, Aug 28, 2025 at 6:00 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
Test-04. pgbench on both side, and max_conflict_retention_duration was tuned
========================================================================
Setup:
-------
Pub --> Sub
- setup is same as Test-03(above)
- Additionally, subscription option max_conflict_retention_duration=60sWorkload:
-------------
- Run default pgbench(read-write) on both Pub and Sub nodes for their
respective pgbench tables
- Started with 15 clients on both sides.
- When conflict_slot.xmin becomes NULL on Sub, pgbench was paused to
let the subscription catch up. Then reduced publisher clients by half
and resumed pgbench. Here, slot.xmin becomes NULL to indicate conflict
retention is stopped under high publisher load but stays non-NULL when
Sub is able to catchup with Pub's load.
- Total duration of pgbench run is 10 minutes (600s).Observations:
------------------
- Without the parallel apply patch, publisher clients reduced from
15->7->3,and finally the retention was not stopped at 3 clients and
slot.xmin remained non-NULL.
- With the parallel apply patch, using 2 workers the subscription
handled up to 7 publisher clients without stopping the conflict
retention.
- With 4+ workers, retention continued for the full 10 minutes and
Sub TPS showed no regression.
I repeated the test with 4 parallel workers over a 12-hour pgbench
run. Retention continued for the full duration without stopping, and
no regression was observed on either publisher or subscriber.
Here is the pgbench result for the run on the publisher node -
number of clients: 15
number of threads: 15
maximum number of tries: 1
duration: 43200 s
number of transactions actually processed: 725093972
number of failed transactions: 0 (0.000%)
latency average = 0.893 ms
latency stddev = 1.176 ms
initial connection time = 8.879 ms
tps = 16784.585718 (without initial connection time)
~~~
Subscriber's TPS = 16668.322700
--
Thanks,
Nisha
On Friday, August 29, 2025 6:28 PM shveta malik <shveta.malik@gmail.com>:
On Fri, Aug 29, 2025 at 11:49 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is the new version patch set which also addressed Shveta's
comments[1].
Thanks for the patch.
On 001 alone, I’m observing a behavior where, if sub1 has stopped retention,
and I then create a new subscription sub2, the worker for
sub2 fails to start successfully. It repeatedly starts and exits, logging the
following message:LOG: logical replication worker for subscription "sub2" will restart because
the option retain_dead_tuples was enabled during startupSame things happen when I disable and re-enable 'retain_dead_tuple' of any
sub once the slot has invalid xmin.
I think this behavior is because slot.xmin is set to an invalid number, and 0001
patch has no slot recovery logic, so even if retentionactive is true, newly
created subscriptions cannot have a valid oldest_nonremovable_xid.
After thinking more, I decided to add slot recovery functionality to 0001 as
well, thus avoiding the need for additional checks here. I also adjusted
the documents accordingly.
Here is the V69 patch set which addressed above comments and the
latest comment from Nisha[1]/messages/by-id/CABdArM7GBa8kXCdOQw4U--tKgapj5j0hAVzL==B3-fkg8Gzmdg@mail.gmail.com.
[1]: /messages/by-id/CABdArM7GBa8kXCdOQw4U--tKgapj5j0hAVzL==B3-fkg8Gzmdg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v69-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v69-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From 08168041a556a9bba2f495a8f076c15a16bef8f1 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v69 3/3] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c77fa0234bb..b47109ab11d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 226421e3dfa..cd54ac7cbf2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1593,7 +1593,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;
@@ -1670,6 +1670,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 6b4a9fb8815..9cfa7c64ced 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.51.0.windows.1
v69-0001-Add-max_retention_duration-option-to-subscriptio.patchapplication/octet-stream; name=v69-0001-Add-max_retention_duration-option-to-subscriptio.patchDownload
From cc368b23bc3a9494a33911fd47034ee39611ea34 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v69 1/2] Add max_retention_duration option to subscriptions.
This commit introduces a new subscription parameter,
max_retention_duration, aimed at mitigating excessive accumulation of dead
tuples when retain_dead_tuples is enabled and the apply worker lags behind
the publisher.
When the time spent advancing a non-removable transaction ID exceeds the
max_retention_duration threshold, the apply worker will stop retaining
conflict detection information. In such cases, the conflict slot's xmin
will be set to InvalidTransactionId, provided that all apply workers
associated with the subscription (with retain_dead_tuples enabled) confirm
the retention duration has been exceeded.
To ensure retention status persists across server restarts, a new column
subretentionactive has been added to the pg_subscription catalog. This
prevents unnecessary reactivation of retention logic after a restart.
The conflict detection slot will not be automatically re-initialized
unless a new subscription is created with retain_dead_tuples = true, or
the user manually re-enables retain_dead_tuples.
A future patch will introduce support for automatic slot re-initialization
once at least one apply worker confirms that the retention duration is
within the configured max_retention_duration.
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
---
doc/src/sgml/catalogs.sgml | 25 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 163 +++++++++----
src/backend/replication/logical/launcher.c | 125 ++++++----
src/backend/replication/logical/worker.c | 260 +++++++++++++++++----
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 16 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 5 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 ++++++++-------
src/test/regress/sql/subscription.sql | 16 ++
18 files changed, 717 insertions(+), 214 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..e9095bedf21 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..12f72ba3167 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..fc314437311 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-retention-duration">
+ <term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration in milliseconds for which this subscription's apply worker
+ is allowed to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled. The default value
+ is <literal>0</literal>, indicating that the information is retained
+ until it is no longer needed for detection purposes.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration has exceeded the
+ <literal>max_retention_duration</literal> set within the corresponding
+ subscription. The retention will not be automatically resumed unless a
+ new subscription is created with <literal>retain_dead_tuples =
+ true</literal>, or the user manually re-enables
+ <literal>retain_dead_tuples</literal>.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions that
+ have a value greater than 0 for this parameter have not exceeded it,
+ or if they set this option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially resulting in incorrect conflict detection.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..b885890de37 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxretention = subform->submaxretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..c77fa0234bb 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxretention, subretentionactive,
+ 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 0d74398faf3..82cf65fae73 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -72,8 +72,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -100,6 +101,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -168,6 +170,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION))
+ opts->maxretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -322,6 +326,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_RETENTION_DURATION;
+ opts->maxretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -579,7 +592,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -646,9 +660,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that system configuration paramters are set appropriately to
+ * support retain_dead_tuples and max_retention_duration.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ (opts.maxretention > 0));
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -692,6 +710,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1197,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_two_phase = false;
bool check_pub_rdt = false;
bool retain_dead_tuples;
+ int max_retention;
+ bool retention_active;
char *origin;
Subscription *sub;
Form_pg_subscription form;
@@ -1205,6 +1229,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = sub->retaindeadtuples;
origin = sub->origin;
+ max_retention = sub->maxretention;
+ retention_active = sub->retentionactive;
/*
* Don't allow non-superuser modification of a subscription with
@@ -1234,7 +1260,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1400,6 +1428,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only if there's a change in
+ * the retain_dead_tuples option value.
+ *
+ * Automatically marking retention as active when
+ * retain_dead_tuples is enabled may not always be ideal,
+ * especially if retention was previously stopped and the
+ * user toggles retain_dead_tuples without adjusting the
+ * publisher workload. However, this behavior provides a
+ * convenient way for users to manually refresh the
+ * retention status. Since retention will be stopped again
+ * unless the publisher workload is reduced, this approach
+ * is acceptable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1416,13 +1467,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
errhint("Try again after some time.")));
- /*
- * Remind the user that enabling subscription will prevent
- * the accumulation of dead tuples.
- */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
-
/*
* Notify the launcher to manage the replication slot for
* conflict detection. This ensures that replication slot
@@ -1435,6 +1479,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ replaces[Anum_pg_subscription_submaxretention - 1] = true;
+
+ max_retention = opts.maxretention;
+ }
+
+ /*
+ * Ensure that system configuration paramters are set
+ * appropriately to support retain_dead_tuples and
+ * max_retention_duration.
+ */
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
+ IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retain_dead_tuples,
+ retention_active,
+ (max_retention > 0));
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1472,9 +1537,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2467,38 +2532,54 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_retention_duration causes no harm,
+ * even when it is ineffective.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool max_retention_set)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
- ereport(ERROR,
- errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
- errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
-
- if (check_guc && !track_commit_timestamp)
- ereport(WARNING,
- errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
- errhint("Consider setting \"%s\" to true.",
- "track_commit_timestamp"));
-
- if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ if (retain_dead_tuples)
+ {
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled && retention_active)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+ }
+ else if (max_retention_set)
+ {
+ ereport(NOTICE,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_dead_tuples") : 0);
+ errmsg("max_retention_duration is ineffective when retain_dead_tuples is disabled"));
+ }
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..add2e2e066c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -43,6 +43,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
@@ -102,7 +103,8 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1181,7 +1184,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool can_advance_xmin = true;
+ bool can_update_xmin = true;
bool retain_dead_tuples = false;
TransactionId xmin = InvalidTransactionId;
@@ -1214,17 +1217,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1232,28 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ can_update_xmin &= sub->enabled;
+
+ /*
+ * Initialize the slot once the subscription activiates
+ * retention.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+ }
}
if (!sub->enabled)
@@ -1254,9 +1268,11 @@ ApplyLauncherMain(Datum main_arg)
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers that enables retain_dead_tuples.
+ * workers.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,12 +1281,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
- can_advance_xmin = false;
+ if (sub->retaindeadtuples && sub->retentionactive)
+ can_update_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1295,7 +1311,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1337,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * Additionally, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to stop retention, the
+ * slot's xmin will be set to InvalidTransactionId allowing the
+ * removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_update_xmin)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1378,7 +1400,15 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Return if the apply worker has stopped retention concurrently.
+ *
+ * Although this function is invoked only when retentionactive is true,
+ * the apply worker might stop retention after the launcher fetches the
+ * retentionactive flag.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1432,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1439,23 +1469,16 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ /* Replication slot must exist but shouldn't be initialized. */
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1475,6 +1498,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..eb2c6dfe734 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,6 +173,14 @@
* Advance the non-removable transaction ID if the current flush location has
* reached or surpassed the last received WAL position.
*
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * configured max_retention_duration. In this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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) ->
@@ -373,7 +381,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION
} RetainDeadTuplesPhase;
/*
@@ -415,6 +424,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +567,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3219,7 +3234,6 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
TimestampTz *delete_time)
{
TransactionId oldestxmin;
- ReplicationSlot *slot;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3229,32 +3243,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
return false;
/*
- * For conflict detection, we use the conflict slot's xmin value instead
- * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
- * a threshold to identify tuples that were recently deleted. These tuples
- * are not visible to concurrent transactions, but we log an
- * update_deleted conflict if such a tuple matches the remote update being
- * applied.
+ * For conflict detection, we use the leader worker's
+ * oldest_nonremovable_xid value instead of invoking
+ * GetOldestNonRemovableTransactionId() or using the conflict detection
+ * slot's xmin. The oldest_nonremovable_xid acts as a threshold to
+ * identify tuples that were recently deleted. These deleted tuples are no
+ * longer visible to concurrent transactions. However, if a remote update
+ * matches such a tuple, we log an update_deleted conflict.
*
- * Although GetOldestNonRemovableTransactionId() can return a value older
- * than the slot's xmin, for our current purpose it is acceptable to treat
- * tuples deleted by transactions prior to slot.xmin as update_missing
- * conflicts.
- *
- * Ideally, we would use oldest_nonremovable_xid, which is directly
- * maintained by the leader apply worker. However, this value is not
- * available to table synchronization or parallel apply workers, making
- * slot.xmin a practical alternative in those contexts.
+ * While GetOldestNonRemovableTransactionId() and slot.xmin may return
+ * transaction IDs older than oldest_nonremovable_xid, for our current
+ * purpose, it is acceptable to treat tuples deleted by transactions prior
+ * to oldest_nonremovable_xid as update_missing conflicts.
*/
- slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+ if (am_leader_apply_worker())
+ {
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+ }
+ else
+ {
+ LogicalRepWorker *leader;
- Assert(slot);
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
- SpinLockAcquire(&slot->mutex);
- oldestxmin = slot->data.xmin;
- SpinLockRelease(&slot->mutex);
+ SpinLockAcquire(&leader->relmutex);
+ oldestxmin = leader->oldest_nonremovable_xid;
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4110,7 +4141,8 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
* Ensure to wake up when it's possible to advance the non-removable
* transaction ID.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (MySubscription->retentionactive &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
@@ -4325,6 +4357,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!MySubscription->retentionactive)
+ return false;
+
return true;
}
@@ -4350,6 +4386,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4507,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4595,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4661,114 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
*/
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return true;
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,22 +4776,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ rdt_data->table_sync_wait_time = 0;
}
/*
* 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.
+ * If there is no activity on the node, we progressively double the interval
+ * used to advance non-removable transaction ID. This helps conserve CPU
+ * and network resources when there's little benefit to frequent updates.
+ *
+ * The interval is capped by the lowest of the following:
+ * - wal_receiver_status_interval (if set),
+ * - a default maximum of 3 minutes,
+ * - max_retention_duration.
*
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * This ensures the interval never exceeds the retention boundary, even if
+ * other limits are higher. Once activity resumes on the node, the interval
+ * is reset to lesser of 100ms and max_retention_duration, allowing timely
+ * advancement of non-removable transaction ID.
*
* 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.
@@ -4651,6 +4823,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
@@ -5458,11 +5634,12 @@ InitializeLogRepWorker(void)
* dropped, a restart is initiated.
*
* The oldest_nonremovable_xid should be initialized only when the
- * retain_dead_tuples is enabled before launching the worker. See
+ * subscription's retention is active before launching the worker. See
* logicalrep_worker_launch.
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5633,8 +5810,9 @@ DisableSubscriptionAndExit(void)
* an error, as verifying commit timestamps is unnecessary in this
* context.
*/
- if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..bea793456f9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxretention = PQfnumber(res, "submaxretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxretention =
+ atoi(PQgetvalue(res, i, i_submaxretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxretention)
+ appendPQExpBuffer(query, ", max_retention_duration = %d", subinfo->submaxretention);
+
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 dde85ed156c..bcc94ff07cc 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..4aa793d7de7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxretention AS \"%s\"\n",
+ gettext_noop("Max retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..6b20a4404b2 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..55cb9b1eefa 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,15 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +145,13 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxretention; /* The maximum duration (in milliseconds) for
+ * which information useful for conflict
+ * detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..fb4e26a51a4 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,9 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool max_retention_set);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c7f1266fc2f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+ERROR: max_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+NOTICE: max_retention_duration is ineffective when retain_dead_tuples 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 dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..ef0c298d2df 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+
+\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;
--
2.51.0.windows.1
v69-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v69-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 75c285139ae6a631d37d81357d40be962e108cab Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 1 Sep 2025 12:40:36 +0800
Subject: [PATCH v69 2/2] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized invalidated slot, if at
least one apply worker has confirmed that the retention duration is now within
the max_retention_duration.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 48 ++++-
src/backend/replication/logical/worker.c | 206 ++++++++++++++++++---
src/include/replication/worker_internal.h | 6 +
5 files changed, 240 insertions(+), 38 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ee22d0fb3ca 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will be automatically resumed once at least
+ one apply worker confirms that the retention duration is within the
+ specified limit, or a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>, or the user manually
+ re-enables <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 82cf65fae73..cc622fce58f 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -854,7 +854,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add2e2e066c..226421e3dfa 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,7 +101,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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
static void init_conflict_slot_xmin(void);
@@ -468,6 +470,7 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->wait_for_initial_xid = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1270,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1382,11 +1383,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * Additionally, if an apply worker has an invalid XID and is requesting to
+ * resume retention, assign the slot's xmin value to it.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool wait_for_xid;
Assert(worker != NULL);
@@ -1398,16 +1404,42 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ wait_for_xid = worker->wait_for_initial_xid;
SpinLockRelease(&worker->relmutex);
/*
- * Return if the apply worker has stopped retention concurrently.
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if
+ * requested. This ensures the apply worker continues to maintain the
+ * oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (wait_for_xid)
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return if the apply worker has stopped retention concurrently and has not
+ * yet resumed.
*
* Although this function is invoked only when retentionactive is true,
* the apply worker might stop retention after the launcher fetches the
* retentionactive flag.
*/
- if (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ if (!can_update_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index eb2c6dfe734..6c5f50a2c07 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION
+ * at any phase if the retention has been stopped, but max_retention_duration
+ * is now set to 0.
+ *
* 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) ->
@@ -382,7 +395,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -569,6 +583,10 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4335,6 +4353,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4357,10 +4382,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4389,6 +4410,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4645,6 +4669,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4675,9 +4711,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4708,10 +4743,16 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
@@ -4721,6 +4762,131 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !MySubscription->retentionactive;
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = false;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4738,26 +4904,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 62ea1a00580..58f2dcc7f0b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -100,6 +100,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker is resuming retention and is waiting
+ * for the launcher to initialize oldest_nonremovable_xid.
+ */
+ bool wait_for_initial_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.51.0.windows.1
On Saturday, August 30, 2025 12:48 PM Nisha Moond <nisha.moond412@gmail.com> wrote:
On Fri, Aug 29, 2025 at 11:49 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is the new version patch set which also addressed Shveta's
comments[1].
Thanks for the patches here, I tested the v68-001 patch alone, please find
review comments -
Thanks for the comments!
1) If a sub is created with retain_dead_tuples=on but disabled, e.g.
postgres=# create subscription sub3 CONNECTION 'dbname=postgres
host=localhost port=8841' PUBLICATION pub3 WITH
(enabled=false,retain_dead_tuples=on);
WARNING: deleted rows to detect conflicts would not be removed until the
subscription is enabled
HINT: Consider setting retain_dead_tuples to false.
NOTICE: created replication slot "sub3" on publisher CREATE
SUBSCRIPTIONIn this case, the conflict slot is not created until the sub is enabled.
I think this is a separate issue that the sub creation command does
not wakeup the launcher to create the slot in time. So, I will prepare a
fix in another thread.
Also, if the
slot already exists but all other subscriptions have stopped retaining
(slot.xmin=NULL), the dead tuple retention will not start until the slot is
recreated.
To me, the above warning seems misleading in this case.2) A similar situation can happen with ALTER SUBSCRIPTION. For example,
consider two subscriptions where retention has stopped for both and slot.xmin
is NULL:subname | subenabled | subretaindeadtuples | submaxretention |
subretentionactive
---------+------------+---------------------+-----------------+---------
---------+------------+---------------------+-----------------+---------
---------+------------+---------------------+-----------------+--
sub2 | t | t | 100 | f
sub1 | t | t | 100 | fpostgres=# select slot_name,active,xmin from pg_replication_slots ;
slot_name | active | xmin
-----------------------+--------+------
pg_conflict_detection | t |If we try to resume retention only for sub1 by toggling retain_dead_tuples:
postgres=# alter subscription sub1 set (retain_dead_tuples =off);
NOTICE: max_retention_duration is ineffective when retain_dead_tuples is
disabled ALTER SUBSCRIPTION postgres=# alter subscription sub1 set
(retain_dead_tuples =on);
NOTICE: deleted rows to detect conflicts would not be removed until the
subscription is enabled ALTER SUBSCRIPTION2a) Here also the retention NOTICE is ambiguous as slot.xmin remains NULL.
Though, the above steps don't strictly follow the docs (i.e.
slot should be recreated to resume the retention), still the notice can be
confusing for users.2b) Also, the retention is not resumed for sub1(expected), but still the
subretentionactive is changed to true.subname | subenabled | subretaindeadtuples | submaxretention |
subretentionactive
---------+------------+---------------------+-----------------+---------
---------+------------+---------------------+-----------------+---------
---------+------------+---------------------+-----------------+--
sub1 | f | t | 100 | t
sub2 | t | t | 100 | fI think we should avoid changing subretentionactive to true in such cases until
the slot is recreated and retention is actually resumed.
Thoughts?
Since I have added slot recovery functionality to 0001, so I think these comments
should also be addressed.
Best Regards,
Hou zj
On Monday, September 1, 2025 12:45 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Friday, August 29, 2025 6:28 PM shveta malik <shveta.malik@gmail.com>:
On Fri, Aug 29, 2025 at 11:49 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:Here is the new version patch set which also addressed Shveta's
comments[1].
Thanks for the patch.
On 001 alone, I’m observing a behavior where, if sub1 has stopped
retention, and I then create a new subscription sub2, the worker for
sub2 fails to start successfully. It repeatedly starts and exits,
logging the following message:LOG: logical replication worker for subscription "sub2" will restart
because the option retain_dead_tuples was enabled during startupSame things happen when I disable and re-enable 'retain_dead_tuple' of
any sub once the slot has invalid xmin.I think this behavior is because slot.xmin is set to an invalid number, and 0001
patch has no slot recovery logic, so even if retentionactive is true, newly created
subscriptions cannot have a valid oldest_nonremovable_xid.After thinking more, I decided to add slot recovery functionality to 0001 as well,
thus avoiding the need for additional checks here. I also adjusted the
documents accordingly.Here is the V69 patch set which addressed above comments and the latest
comment from Nisha[1].
I reviewed the patch internally and tweaked a small detail of the apply worker
to reduce the waiting time in the main loop when max_retention_duration is
defined (set wait_time = min(wait_time, max_retention_duration)). Also, I added
a simple test in 035_conflicts.pl of 0001 to verify the new sub option.
Here is V70 patch set.
Best Regards,
Hou zj
Attachments:
v70-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v70-0003-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From b13962ec5cb9de8fecefec23bca26b24ba134388 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v70 3/3] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 10 +++++++++-
6 files changed, 46 insertions(+), 8 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c77fa0234bb..b47109ab11d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 226421e3dfa..cd54ac7cbf2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1593,7 +1593,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;
@@ -1670,6 +1670,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 51b23a39fa9..d1b76d9f378 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -214,6 +214,10 @@ ok( $node_B->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node B");
+my $result = $node_B->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_BA';");
+is($result, qq(t), 'worker on node B retains conflict information');
+
##################################################
# Check that the retain_dead_tuples option can be enabled only for disabled
# subscriptions. Validate the NOTICE message during the subscription DDL, and
@@ -254,6 +258,10 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is valid on Node A");
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains conflict information');
+
##################################################
# Check the WARNING when changing the origin to ANY, if retain_dead_tuples is
# enabled. This warns of the possibility of receiving changes from origins
@@ -281,7 +289,7 @@ $node_A->psql('postgres',
$node_A->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1), (2, 2);");
$node_A->wait_for_catchup($subname_BA);
-my $result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
+$result = $node_B->safe_psql('postgres', "SELECT * FROM tab;");
is($result, qq(1|1
2|2), 'check replicated insert on node B');
--
2.51.0.windows.1
v70-0001-Add-max_retention_duration-option-to-subscriptio.patchapplication/octet-stream; name=v70-0001-Add-max_retention_duration-option-to-subscriptio.patchDownload
From 0e6bd60f9527bf500402b86954a6c7b1b07647e0 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:06:10 +0800
Subject: [PATCH v70 1/3] Add max_retention_duration option to subscriptions.
This commit introduces a new subscription parameter,
max_retention_duration, aimed at mitigating excessive accumulation of dead
tuples when retain_dead_tuples is enabled and the apply worker lags behind
the publisher.
When the time spent advancing a non-removable transaction ID exceeds the
max_retention_duration threshold, the apply worker will stop retaining
conflict detection information. In such cases, the conflict slot's xmin
will be set to InvalidTransactionId, provided that all apply workers
associated with the subscription (with retain_dead_tuples enabled) confirm
the retention duration has been exceeded.
To ensure retention status persists across server restarts, a new column
subretentionactive has been added to the pg_subscription catalog. This
prevents unnecessary reactivation of retention logic after a restart.
The conflict detection slot will not be automatically re-initialized
unless a new subscription is created with retain_dead_tuples = true, or
the user manually re-enables retain_dead_tuples.
A future patch will introduce support for automatic slot re-initialization
once at least one apply worker confirms that the retention duration is
within the configured max_retention_duration.
Author: Zhijie Hou <houzj.fnst@fujitsu.com>
Reviewed-by: shveta malik <shveta.malik@gmail.com>
Reviewed-by: Nisha Moond <nisha.moond412@gmail.com>
Reviewed-by: Masahiko Sawada <sawada.mshk@gmail.com>
Reviewed-by: Dilip Kumar <dilipbalaut@gmail.com>
Reviewed-by: Amit Kapila <amit.kapila16@gmail.com>
---
doc/src/sgml/catalogs.sgml | 25 ++
doc/src/sgml/ref/alter_subscription.sgml | 5 +-
doc/src/sgml/ref/create_subscription.sgml | 43 +++-
src/backend/catalog/pg_subscription.c | 41 ++++
src/backend/catalog/system_views.sql | 4 +-
src/backend/commands/subscriptioncmds.c | 163 +++++++++----
src/backend/replication/logical/launcher.c | 125 ++++++----
src/backend/replication/logical/worker.c | 266 +++++++++++++++++----
src/bin/pg_dump/pg_dump.c | 18 +-
src/bin/pg_dump/pg_dump.h | 1 +
src/bin/psql/describe.c | 12 +-
src/bin/psql/tab-complete.in.c | 6 +-
src/include/catalog/pg_subscription.h | 16 ++
src/include/catalog/pg_subscription_rel.h | 2 +
src/include/commands/subscriptioncmds.h | 5 +-
src/include/replication/worker_internal.h | 3 +
src/test/regress/expected/subscription.out | 186 +++++++-------
src/test/regress/sql/subscription.sql | 16 ++
src/test/subscription/t/035_conflicts.pl | 53 ++++
19 files changed, 775 insertions(+), 215 deletions(-)
diff --git a/doc/src/sgml/catalogs.sgml b/doc/src/sgml/catalogs.sgml
index da8a7882580..e9095bedf21 100644
--- a/doc/src/sgml/catalogs.sgml
+++ b/doc/src/sgml/catalogs.sgml
@@ -8094,6 +8094,31 @@ SCRAM-SHA-256$<replaceable><iteration count></replaceable>:<replaceable>&l
</para></entry>
</row>
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>submaxretention</structfield> <type>int4</type>
+ </para>
+ <para>
+ The maximum duration (in milliseconds) for which information (e.g., dead
+ tuples, commit timestamps, and origins) useful for conflict detection can
+ be retained.
+ </para></entry>
+ </row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>subretentionactive</structfield> <type>bool</type>
+ </para>
+ <para>
+ The retention status of information (e.g., dead tuples, commit
+ timestamps, and origins) useful for conflict detection. True if
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled, and the retention duration has not exceeded
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>,
+ when defined.
+ </para></entry>
+ </row>
+
<row>
<entry role="catalog_table_entry"><para role="column_definition">
<structfield>subconninfo</structfield> <type>text</type>
diff --git a/doc/src/sgml/ref/alter_subscription.sgml b/doc/src/sgml/ref/alter_subscription.sgml
index d48cdc76bd3..12f72ba3167 100644
--- a/doc/src/sgml/ref/alter_subscription.sgml
+++ b/doc/src/sgml/ref/alter_subscription.sgml
@@ -236,8 +236,9 @@ ALTER SUBSCRIPTION <replaceable class="parameter">name</replaceable> RENAME TO <
<link linkend="sql-createsubscription-params-with-run-as-owner"><literal>run_as_owner</literal></link>,
<link linkend="sql-createsubscription-params-with-origin"><literal>origin</literal></link>,
<link linkend="sql-createsubscription-params-with-failover"><literal>failover</literal></link>,
- <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>, and
- <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>.
+ <link linkend="sql-createsubscription-params-with-two-phase"><literal>two_phase</literal></link>,
+ <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>, and
+ <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>.
Only a superuser can set <literal>password_required = false</literal>.
</para>
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index 247c5bd2604..fc314437311 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -448,7 +448,7 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
If set to <literal>true</literal>, the detection of
<xref linkend="conflict-update-deleted"/> is enabled, and a physical
replication slot named <quote><literal>pg_conflict_detection</literal></quote>
- created on the subscriber to prevent the information for detecting
+ is created on the subscriber to prevent the information for detecting
conflicts from being removed.
</para>
@@ -521,6 +521,47 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
</para>
</listitem>
</varlistentry>
+
+ <varlistentry id="sql-createsubscription-params-with-max-retention-duration">
+ <term><literal>max_retention_duration</literal> (<type>integer</type>)</term>
+ <listitem>
+ <para>
+ Maximum duration in milliseconds for which this subscription's apply worker
+ is allowed to retain the information useful for conflict detection when
+ <literal>retain_dead_tuples</literal> is enabled. The default value
+ is <literal>0</literal>, indicating that the information is retained
+ until it is no longer needed for detection purposes.
+ </para>
+ <para>
+ The information useful for conflict detection is no longer retained if
+ all apply workers associated with the subscriptions, where
+ <literal>retain_dead_tuples</literal> is enabled, confirm that the
+ retention duration has exceeded the
+ <literal>max_retention_duration</literal> set within the corresponding
+ subscription. The retention will not be automatically resumed unless a
+ new subscription is created with <literal>retain_dead_tuples =
+ true</literal>, or the user manually re-enables
+ <literal>retain_dead_tuples</literal>.
+ </para>
+ <para>
+ Note that overall retention will not stop if other subscriptions that
+ have a value greater than 0 for this parameter have not exceeded it,
+ or if they set this option to 0.
+ </para>
+ <para>
+ This option is effective only when
+ <literal>retain_conflict_info</literal> is enabled and the apply
+ worker associated with the subscription is active.
+ </para>
+ <warning>
+ <para>
+ Note that setting a non-zero value for this option could lead to
+ information for conflict detection being removed prematurely,
+ potentially resulting in incorrect conflict detection.
+ </para>
+ </warning>
+ </listitem>
+ </varlistentry>
</variablelist></para>
</listitem>
diff --git a/src/backend/catalog/pg_subscription.c b/src/backend/catalog/pg_subscription.c
index 244acf52f36..b885890de37 100644
--- a/src/backend/catalog/pg_subscription.c
+++ b/src/backend/catalog/pg_subscription.c
@@ -104,6 +104,8 @@ GetSubscription(Oid subid, bool missing_ok)
sub->runasowner = subform->subrunasowner;
sub->failover = subform->subfailover;
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->maxretention = subform->submaxretention;
+ sub->retentionactive = subform->subretentionactive;
/* Get conninfo */
datum = SysCacheGetAttrNotNull(SUBSCRIPTIONOID,
@@ -598,3 +600,42 @@ GetSubscriptionRelations(Oid subid, bool not_ready)
return res;
}
+
+/*
+ * Update the dead tuple retention status for the given subscription.
+ */
+void
+UpdateDeadTupleRetentionStatus(Oid subid, bool active)
+{
+ Relation rel;
+ bool nulls[Natts_pg_subscription];
+ bool replaces[Natts_pg_subscription];
+ Datum values[Natts_pg_subscription];
+ HeapTuple tup;
+
+ /* Look up the subscription in the catalog */
+ rel = table_open(SubscriptionRelationId, RowExclusiveLock);
+ tup = SearchSysCacheCopy1(SUBSCRIPTIONOID, ObjectIdGetDatum(subid));
+
+ if (!HeapTupleIsValid(tup))
+ elog(ERROR, "cache lookup failed for subscription %u", subid);
+
+ LockSharedObject(SubscriptionRelationId, subid, 0, AccessShareLock);
+
+ /* Form a new tuple. */
+ memset(values, 0, sizeof(values));
+ memset(nulls, false, sizeof(nulls));
+ memset(replaces, false, sizeof(replaces));
+
+ /* Set the subscription to disabled. */
+ values[Anum_pg_subscription_subretentionactive - 1] = active;
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ /* Update the catalog */
+ tup = heap_modify_tuple(tup, RelationGetDescr(rel), values, nulls,
+ replaces);
+ CatalogTupleUpdate(rel, &tup->t_self, tup);
+ heap_freetuple(tup);
+
+ table_close(rel, NoLock);
+}
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 1b3c5a55882..c77fa0234bb 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -1389,8 +1389,8 @@ REVOKE ALL ON pg_subscription FROM public;
GRANT SELECT (oid, subdbid, subskiplsn, subname, subowner, subenabled,
subbinary, substream, subtwophasestate, subdisableonerr,
subpasswordrequired, subrunasowner, subfailover,
- subretaindeadtuples, subslotname, subsynccommit,
- subpublications, suborigin)
+ subretaindeadtuples, submaxretention, subretentionactive,
+ 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 0d74398faf3..82cf65fae73 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -72,8 +72,9 @@
#define SUBOPT_RUN_AS_OWNER 0x00001000
#define SUBOPT_FAILOVER 0x00002000
#define SUBOPT_RETAIN_DEAD_TUPLES 0x00004000
-#define SUBOPT_LSN 0x00008000
-#define SUBOPT_ORIGIN 0x00010000
+#define SUBOPT_MAX_RETENTION_DURATION 0x00008000
+#define SUBOPT_LSN 0x00010000
+#define SUBOPT_ORIGIN 0x00020000
/* check if the 'val' has 'bits' set */
#define IsSet(val, bits) (((val) & (bits)) == (bits))
@@ -100,6 +101,7 @@ typedef struct SubOpts
bool runasowner;
bool failover;
bool retaindeadtuples;
+ int32 maxretention;
char *origin;
XLogRecPtr lsn;
} SubOpts;
@@ -168,6 +170,8 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->failover = false;
if (IsSet(supported_opts, SUBOPT_RETAIN_DEAD_TUPLES))
opts->retaindeadtuples = false;
+ if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION))
+ opts->maxretention = 0;
if (IsSet(supported_opts, SUBOPT_ORIGIN))
opts->origin = pstrdup(LOGICALREP_ORIGIN_ANY);
@@ -322,6 +326,15 @@ parse_subscription_options(ParseState *pstate, List *stmt_options,
opts->specified_opts |= SUBOPT_RETAIN_DEAD_TUPLES;
opts->retaindeadtuples = defGetBoolean(defel);
}
+ else if (IsSet(supported_opts, SUBOPT_MAX_RETENTION_DURATION) &&
+ strcmp(defel->defname, "max_retention_duration") == 0)
+ {
+ if (IsSet(opts->specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ errorConflictingDefElem(defel, pstate);
+
+ opts->specified_opts |= SUBOPT_MAX_RETENTION_DURATION;
+ opts->maxretention = defGetInt32(defel);
+ }
else if (IsSet(supported_opts, SUBOPT_ORIGIN) &&
strcmp(defel->defname, "origin") == 0)
{
@@ -579,7 +592,8 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
SUBOPT_STREAMING | SUBOPT_TWOPHASE_COMMIT |
SUBOPT_DISABLE_ON_ERR | SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION | SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options, supported_opts, &opts);
/*
@@ -646,9 +660,13 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
stmt->subname)));
}
- /* Ensure that we can enable retain_dead_tuples */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !opts.enabled, WARNING);
+ /*
+ * Ensure that system configuration paramters are set appropriately to
+ * support retain_dead_tuples and max_retention_duration.
+ */
+ CheckSubDeadTupleRetention(true, !opts.enabled, WARNING,
+ opts.retaindeadtuples, opts.retaindeadtuples,
+ (opts.maxretention > 0));
if (!IsSet(opts.specified_opts, SUBOPT_SLOT_NAME) &&
opts.slot_name == NULL)
@@ -692,6 +710,10 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
values[Anum_pg_subscription_subfailover - 1] = BoolGetDatum(opts.failover);
values[Anum_pg_subscription_subretaindeadtuples - 1] =
BoolGetDatum(opts.retaindeadtuples);
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ Int32GetDatum(opts.retaindeadtuples);
values[Anum_pg_subscription_subconninfo - 1] =
CStringGetTextDatum(conninfo);
if (opts.slot_name)
@@ -1175,6 +1197,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
bool update_two_phase = false;
bool check_pub_rdt = false;
bool retain_dead_tuples;
+ int max_retention;
+ bool retention_active;
char *origin;
Subscription *sub;
Form_pg_subscription form;
@@ -1205,6 +1229,8 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = sub->retaindeadtuples;
origin = sub->origin;
+ max_retention = sub->maxretention;
+ retention_active = sub->retentionactive;
/*
* Don't allow non-superuser modification of a subscription with
@@ -1234,7 +1260,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
SUBOPT_DISABLE_ON_ERR |
SUBOPT_PASSWORD_REQUIRED |
SUBOPT_RUN_AS_OWNER | SUBOPT_FAILOVER |
- SUBOPT_RETAIN_DEAD_TUPLES | SUBOPT_ORIGIN);
+ SUBOPT_RETAIN_DEAD_TUPLES |
+ SUBOPT_MAX_RETENTION_DURATION |
+ SUBOPT_ORIGIN);
parse_subscription_options(pstate, stmt->options,
supported_opts, &opts);
@@ -1400,6 +1428,29 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
BoolGetDatum(opts.retaindeadtuples);
replaces[Anum_pg_subscription_subretaindeadtuples - 1] = true;
+ /*
+ * Update the retention status only if there's a change in
+ * the retain_dead_tuples option value.
+ *
+ * Automatically marking retention as active when
+ * retain_dead_tuples is enabled may not always be ideal,
+ * especially if retention was previously stopped and the
+ * user toggles retain_dead_tuples without adjusting the
+ * publisher workload. However, this behavior provides a
+ * convenient way for users to manually refresh the
+ * retention status. Since retention will be stopped again
+ * unless the publisher workload is reduced, this approach
+ * is acceptable for now.
+ */
+ if (opts.retaindeadtuples != sub->retaindeadtuples)
+ {
+ values[Anum_pg_subscription_subretentionactive - 1] =
+ BoolGetDatum(opts.retaindeadtuples);
+ replaces[Anum_pg_subscription_subretentionactive - 1] = true;
+
+ retention_active = opts.retaindeadtuples;
+ }
+
CheckAlterSubOption(sub, "retain_dead_tuples", false, isTopLevel);
/*
@@ -1416,13 +1467,6 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
errmsg("cannot alter retain_dead_tuples when logical replication worker is still running"),
errhint("Try again after some time.")));
- /*
- * Remind the user that enabling subscription will prevent
- * the accumulation of dead tuples.
- */
- if (opts.retaindeadtuples)
- CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE);
-
/*
* Notify the launcher to manage the replication slot for
* conflict detection. This ensures that replication slot
@@ -1435,6 +1479,27 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
retain_dead_tuples = opts.retaindeadtuples;
}
+ if (IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ {
+ values[Anum_pg_subscription_submaxretention - 1] =
+ Int32GetDatum(opts.maxretention);
+ replaces[Anum_pg_subscription_submaxretention - 1] = true;
+
+ max_retention = opts.maxretention;
+ }
+
+ /*
+ * Ensure that system configuration paramters are set
+ * appropriately to support retain_dead_tuples and
+ * max_retention_duration.
+ */
+ if (IsSet(opts.specified_opts, SUBOPT_RETAIN_DEAD_TUPLES) ||
+ IsSet(opts.specified_opts, SUBOPT_MAX_RETENTION_DURATION))
+ CheckSubDeadTupleRetention(true, !sub->enabled, NOTICE,
+ retain_dead_tuples,
+ retention_active,
+ (max_retention > 0));
+
if (IsSet(opts.specified_opts, SUBOPT_ORIGIN))
{
values[Anum_pg_subscription_suborigin - 1] =
@@ -1472,9 +1537,9 @@ AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt,
* subscription in case it was disabled after creation. See
* comments atop CheckSubDeadTupleRetention() for details.
*/
- if (sub->retaindeadtuples)
- CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
- WARNING);
+ CheckSubDeadTupleRetention(opts.enabled, !opts.enabled,
+ WARNING, sub->retaindeadtuples,
+ sub->retentionactive, false);
values[Anum_pg_subscription_subenabled - 1] =
BoolGetDatum(opts.enabled);
@@ -2467,38 +2532,54 @@ check_pub_dead_tuple_retention(WalReceiverConn *wrconn)
* this setting can be adjusted after subscription creation. Without it, the
* apply worker will simply skip conflict detection.
*
- * Issue a WARNING or NOTICE if the subscription is disabled. Do not raise an
- * ERROR since users can only modify retain_dead_tuples for disabled
- * subscriptions. And as long as the subscription is enabled promptly, it will
- * not pose issues.
+ * Issue a WARNING or NOTICE if the subscription is disabled and the retention
+ * is active. Do not raise an ERROR since users can only modify
+ * retain_dead_tuples for disabled subscriptions. And as long as the
+ * subscription is enabled promptly, it will not pose issues.
+ *
+ * Issue a NOTICE to inform users that max_retention_duration is
+ * ineffective when retain_dead_tuples is disabled for a subscription. An ERROR
+ * is not issued because setting max_retention_duration causes no harm,
+ * even when it is ineffective.
*/
void
CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled)
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples, bool retention_active,
+ bool max_retention_set)
{
Assert(elevel_for_sub_disabled == NOTICE ||
elevel_for_sub_disabled == WARNING);
- if (check_guc && wal_level < WAL_LEVEL_REPLICA)
- ereport(ERROR,
- errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
- errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
- errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
-
- if (check_guc && !track_commit_timestamp)
- ereport(WARNING,
- errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
- errhint("Consider setting \"%s\" to true.",
- "track_commit_timestamp"));
-
- if (sub_disabled)
- ereport(elevel_for_sub_disabled,
+ if (retain_dead_tuples)
+ {
+ if (check_guc && wal_level < WAL_LEVEL_REPLICA)
+ ereport(ERROR,
+ errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("\"wal_level\" is insufficient to create the replication slot required by retain_dead_tuples"),
+ errhint("\"wal_level\" must be set to \"replica\" or \"logical\" at server start."));
+
+ if (check_guc && !track_commit_timestamp)
+ ereport(WARNING,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("commit timestamp and origin data required for detecting conflicts won't be retained"),
+ errhint("Consider setting \"%s\" to true.",
+ "track_commit_timestamp"));
+
+ if (sub_disabled && retention_active)
+ ereport(elevel_for_sub_disabled,
+ errcode(ERRCODE_INVALID_PARAMETER_VALUE),
+ errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
+ (elevel_for_sub_disabled > NOTICE)
+ ? errhint("Consider setting %s to false.",
+ "retain_dead_tuples") : 0);
+ }
+ else if (max_retention_set)
+ {
+ ereport(NOTICE,
errcode(ERRCODE_INVALID_PARAMETER_VALUE),
- errmsg("deleted rows to detect conflicts would not be removed until the subscription is enabled"),
- (elevel_for_sub_disabled > NOTICE)
- ? errhint("Consider setting %s to false.",
- "retain_dead_tuples") : 0);
+ errmsg("max_retention_duration is ineffective when retain_dead_tuples is disabled"));
+ }
}
/*
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 37377f7eb63..add2e2e066c 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -43,6 +43,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
@@ -102,7 +103,8 @@ static void ApplyLauncherSetWorkerStartTime(Oid subid, TimestampTz start_time);
static TimestampTz ApplyLauncherGetWorkerStartTime(Oid subid);
static void compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
-static void advance_conflict_slot_xmin(TransactionId new_xmin);
+static void update_conflict_slot_xmin(TransactionId new_xmin);
+static void init_conflict_slot_xmin(void);
/*
@@ -152,6 +154,7 @@ get_subscription_list(void)
sub->enabled = subform->subenabled;
sub->name = pstrdup(NameStr(subform->subname));
sub->retaindeadtuples = subform->subretaindeadtuples;
+ sub->retentionactive = subform->subretentionactive;
/* We don't fill fields we are not interested in. */
res = lappend(res, sub);
@@ -1181,7 +1184,7 @@ ApplyLauncherMain(Datum main_arg)
MemoryContext subctx;
MemoryContext oldctx;
long wait_time = DEFAULT_NAPTIME_PER_CYCLE;
- bool can_advance_xmin = true;
+ bool can_update_xmin = true;
bool retain_dead_tuples = false;
TransactionId xmin = InvalidTransactionId;
@@ -1214,17 +1217,6 @@ ApplyLauncherMain(Datum main_arg)
{
retain_dead_tuples = true;
- /*
- * Can't advance xmin of the slot unless all the subscriptions
- * with retain_dead_tuples are enabled. This is required to
- * ensure that we don't advance the xmin of
- * CONFLICT_DETECTION_SLOT if one of the subscriptions is not
- * enabled. Otherwise, we won't be able to detect conflicts
- * reliably for such a subscription even though it has set the
- * retain_dead_tuples option.
- */
- can_advance_xmin &= sub->enabled;
-
/*
* Create a replication slot to retain information necessary
* for conflict detection such as dead tuples, commit
@@ -1240,6 +1232,28 @@ ApplyLauncherMain(Datum main_arg)
* subscription was enabled.
*/
CreateConflictDetectionSlot();
+
+ if (sub->retentionactive)
+ {
+ /*
+ * Can't advance xmin of the slot unless all the
+ * subscriptions actively retaining dead tuples are
+ * enabled. This is required to ensure that we don't
+ * advance the xmin of CONFLICT_DETECTION_SLOT if one of
+ * the subscriptions is not enabled. Otherwise, we won't
+ * be able to detect conflicts reliably for such a
+ * subscription even though it has set the
+ * retain_dead_tuples option.
+ */
+ can_update_xmin &= sub->enabled;
+
+ /*
+ * Initialize the slot once the subscription activiates
+ * retention.
+ */
+ if (!TransactionIdIsValid(MyReplicationSlot->data.xmin))
+ init_conflict_slot_xmin();
+ }
}
if (!sub->enabled)
@@ -1254,9 +1268,11 @@ ApplyLauncherMain(Datum main_arg)
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers that enables retain_dead_tuples.
+ * workers.
*/
- if (sub->retaindeadtuples && can_advance_xmin)
+ if (sub->retaindeadtuples &&
+ sub->retentionactive &&
+ can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
/* worker is running already */
@@ -1265,12 +1281,12 @@ ApplyLauncherMain(Datum main_arg)
/*
* Can't advance xmin of the slot unless all the workers
- * corresponding to subscriptions with retain_dead_tuples are
- * running, disabling the further computation of the minimum
+ * corresponding to subscriptions actively retaining dead tuples
+ * are running, disabling the further computation of the minimum
* nonremovable xid.
*/
- if (sub->retaindeadtuples)
- can_advance_xmin = false;
+ if (sub->retaindeadtuples && sub->retentionactive)
+ can_update_xmin = false;
/*
* If the worker is eligible to start now, launch it. Otherwise,
@@ -1295,7 +1311,8 @@ ApplyLauncherMain(Datum main_arg)
sub->dbid, sub->oid, sub->name,
sub->owner, InvalidOid,
DSM_HANDLE_INVALID,
- sub->retaindeadtuples))
+ sub->retaindeadtuples &&
+ sub->retentionactive))
{
/*
* We get here either if we failed to launch a worker
@@ -1320,13 +1337,18 @@ ApplyLauncherMain(Datum main_arg)
* that requires us to retain dead tuples. Otherwise, if required,
* advance the slot's xmin to protect dead tuples required for the
* conflict detection.
+ *
+ * Additionally, if all apply workers for subscriptions with
+ * retain_dead_tuples enabled have requested to stop retention, the
+ * slot's xmin will be set to InvalidTransactionId allowing the
+ * removal of dead tuples.
*/
if (MyReplicationSlot)
{
if (!retain_dead_tuples)
ReplicationSlotDropAcquired();
- else if (can_advance_xmin)
- advance_conflict_slot_xmin(xmin);
+ else if (can_update_xmin)
+ update_conflict_slot_xmin(xmin);
}
/* Switch back to original memory context. */
@@ -1378,7 +1400,15 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
nonremovable_xid = worker->oldest_nonremovable_xid;
SpinLockRelease(&worker->relmutex);
- Assert(TransactionIdIsValid(nonremovable_xid));
+ /*
+ * Return if the apply worker has stopped retention concurrently.
+ *
+ * Although this function is invoked only when retentionactive is true,
+ * the apply worker might stop retention after the launcher fetches the
+ * retentionactive flag.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
if (!TransactionIdIsValid(*xmin) ||
TransactionIdPrecedes(nonremovable_xid, *xmin))
@@ -1402,17 +1432,17 @@ acquire_conflict_slot_if_exists(void)
}
/*
- * Advance the xmin the replication slot used to retain information required
+ * Update the xmin the replication slot used to retain information required
* for conflict detection.
*/
static void
-advance_conflict_slot_xmin(TransactionId new_xmin)
+update_conflict_slot_xmin(TransactionId new_xmin)
{
Assert(MyReplicationSlot);
- Assert(TransactionIdIsValid(new_xmin));
- Assert(TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
+ Assert(!TransactionIdIsValid(new_xmin) ||
+ TransactionIdPrecedesOrEquals(MyReplicationSlot->data.xmin, new_xmin));
- /* Return if the xmin value of the slot cannot be advanced */
+ /* Return if the xmin value of the slot cannot be updated */
if (TransactionIdEquals(MyReplicationSlot->data.xmin, new_xmin))
return;
@@ -1439,23 +1469,16 @@ advance_conflict_slot_xmin(TransactionId new_xmin)
}
/*
- * Create and acquire the replication slot used to retain information for
- * conflict detection, if not yet.
+ * Initialize the xmin for the conflict detection slot.
*/
-void
-CreateConflictDetectionSlot(void)
+static void
+init_conflict_slot_xmin(void)
{
TransactionId xmin_horizon;
- /* Exit early, if the replication slot is already created and acquired */
- if (MyReplicationSlot)
- return;
-
- ereport(LOG,
- errmsg("creating replication conflict detection slot"));
-
- ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
- false, false);
+ /* Replication slot must exist but shouldn't be initialized. */
+ Assert(MyReplicationSlot &&
+ !TransactionIdIsValid(MyReplicationSlot->data.xmin));
LWLockAcquire(ProcArrayLock, LW_EXCLUSIVE);
@@ -1475,6 +1498,26 @@ CreateConflictDetectionSlot(void)
ReplicationSlotSave();
}
+/*
+ * Create and acquire the replication slot used to retain information for
+ * conflict detection, if not yet.
+ */
+void
+CreateConflictDetectionSlot(void)
+{
+ /* Exit early, if the replication slot is already created and acquired */
+ if (MyReplicationSlot)
+ return;
+
+ ereport(LOG,
+ errmsg("creating replication conflict detection slot"));
+
+ ReplicationSlotCreate(CONFLICT_DETECTION_SLOT, false, RS_PERSISTENT, false,
+ false, false);
+
+ init_conflict_slot_xmin();
+}
+
/*
* Is current process the logical replication launcher?
*/
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 22ad9051db3..f5f9183bdaf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -173,6 +173,14 @@
* Advance the non-removable transaction ID if the current flush location has
* reached or surpassed the last received WAL position.
*
+ * - RDT_STOP_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the wait time in either the
+ * RDT_WAIT_FOR_PUBLISHER_STATUS or RDT_WAIT_FOR_LOCAL_FLUSH phase exceeds
+ * configured max_retention_duration. In this phase,
+ * pg_subscription.subretentionactive is updated to false within a new
+ * transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
+ *
* 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) ->
@@ -373,7 +381,8 @@ typedef enum
RDT_GET_CANDIDATE_XID,
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
- RDT_WAIT_FOR_LOCAL_FLUSH
+ RDT_WAIT_FOR_LOCAL_FLUSH,
+ RDT_STOP_CONFLICT_INFO_RETENTION
} RetainDeadTuplesPhase;
/*
@@ -415,6 +424,9 @@ typedef struct RetainDeadTuplesData
* updated in final phase
* (RDT_WAIT_FOR_LOCAL_FLUSH) */
+ long table_sync_wait_time; /* time spent waiting for table sync
+ * to finish */
+
/*
* The following fields are used to determine the timing for the next
* round of transaction ID advancement.
@@ -555,6 +567,9 @@ static void request_publisher_status(RetainDeadTuplesData *rdt_data);
static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
bool status_received);
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
+static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -3219,7 +3234,6 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
TimestampTz *delete_time)
{
TransactionId oldestxmin;
- ReplicationSlot *slot;
/*
* Return false if either dead tuples are not retained or commit timestamp
@@ -3229,32 +3243,49 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
return false;
/*
- * For conflict detection, we use the conflict slot's xmin value instead
- * of invoking GetOldestNonRemovableTransactionId(). The slot.xmin acts as
- * a threshold to identify tuples that were recently deleted. These tuples
- * are not visible to concurrent transactions, but we log an
- * update_deleted conflict if such a tuple matches the remote update being
- * applied.
+ * For conflict detection, we use the leader worker's
+ * oldest_nonremovable_xid value instead of invoking
+ * GetOldestNonRemovableTransactionId() or using the conflict detection
+ * slot's xmin. The oldest_nonremovable_xid acts as a threshold to
+ * identify tuples that were recently deleted. These deleted tuples are no
+ * longer visible to concurrent transactions. However, if a remote update
+ * matches such a tuple, we log an update_deleted conflict.
*
- * Although GetOldestNonRemovableTransactionId() can return a value older
- * than the slot's xmin, for our current purpose it is acceptable to treat
- * tuples deleted by transactions prior to slot.xmin as update_missing
- * conflicts.
- *
- * Ideally, we would use oldest_nonremovable_xid, which is directly
- * maintained by the leader apply worker. However, this value is not
- * available to table synchronization or parallel apply workers, making
- * slot.xmin a practical alternative in those contexts.
+ * While GetOldestNonRemovableTransactionId() and slot.xmin may return
+ * transaction IDs older than oldest_nonremovable_xid, for our current
+ * purpose, it is acceptable to treat tuples deleted by transactions prior
+ * to oldest_nonremovable_xid as update_missing conflicts.
*/
- slot = SearchNamedReplicationSlot(CONFLICT_DETECTION_SLOT, true);
+ if (am_leader_apply_worker())
+ {
+ oldestxmin = MyLogicalRepWorker->oldest_nonremovable_xid;
+ }
+ else
+ {
+ LogicalRepWorker *leader;
- Assert(slot);
+ /*
+ * Obtain the information from the leader apply worker as only the
+ * leader manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
+ InvalidOid, false);
- SpinLockAcquire(&slot->mutex);
- oldestxmin = slot->data.xmin;
- SpinLockRelease(&slot->mutex);
+ SpinLockAcquire(&leader->relmutex);
+ oldestxmin = leader->oldest_nonremovable_xid;
+ SpinLockRelease(&leader->relmutex);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
- Assert(TransactionIdIsValid(oldestxmin));
+ /*
+ * Return false if the leader apply worker has stopped retaining
+ * information for detecting conflicts. This implies that update_deleted
+ * can no longer be reliably detected.
+ */
+ if (!TransactionIdIsValid(oldestxmin))
+ return false;
if (OidIsValid(localidxoid) &&
IsIndexUsableForFindingDeletedTuple(localidxoid, oldestxmin))
@@ -4108,11 +4139,16 @@ LogicalRepApplyLoop(XLogRecPtr last_received)
/*
* Ensure to wake up when it's possible to advance the non-removable
- * transaction ID.
+ * transaction ID, or when the retention duration may have exceeded
+ * max_retention_duration.
*/
- if (rdt_data.phase == RDT_GET_CANDIDATE_XID &&
+ if (MySubscription->retentionactive &&
+ rdt_data.phase == RDT_GET_CANDIDATE_XID &&
rdt_data.xid_advance_interval)
wait_time = Min(wait_time, rdt_data.xid_advance_interval);
+ else if (MySubscription->retentionactive &&
+ MySubscription->maxretention > 0)
+ wait_time = Min(wait_time, MySubscription->maxretention);
rc = WaitLatchOrSocket(MyLatch,
WL_SOCKET_READABLE | WL_LATCH_SET |
@@ -4325,6 +4361,10 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
+ /* No need to advance if we have already stopped retaining */
+ if (!MySubscription->retentionactive)
+ return false;
+
return true;
}
@@ -4350,6 +4390,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_WAIT_FOR_LOCAL_FLUSH:
wait_for_local_flush(rdt_data);
break;
+ case RDT_STOP_CONFLICT_INFO_RETENTION:
+ stop_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4468,6 +4511,13 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
if (!status_received)
return;
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
+ return;
+
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4549,6 +4599,27 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* have a WAL position greater than the rdt_data->remote_lsn.
*/
if (!AllTablesyncsReady())
+ {
+ TimestampTz now;
+
+ now = rdt_data->last_recv_time
+ ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
+ /*
+ * Record the time spent waiting for table sync, it is needed for the
+ * timeout check in should_stop_conflict_info_retention().
+ */
+ rdt_data->table_sync_wait_time =
+ TimestampDifferenceMilliseconds(rdt_data->candidate_xid_time, now);
+
+ return;
+ }
+
+ /*
+ * We don't need to maintain oldest_nonremovable_xid if we decide to stop
+ * retaining conflict information for this worker.
+ */
+ if (should_stop_conflict_info_retention(rdt_data))
return;
/*
@@ -4594,12 +4665,114 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether conflict information retention should be stopped due to
+ * exceeding the maximum wait time (max_retention_duration).
+ *
+ * If retention should be stopped, transition to the
+ * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
+ * false.
+ *
+ * Note: Retention won't be resumed automatically. The user must manually
+ * disable retain_dead_tuples and re-enable it after confirming that the
+ * replication slot maintained by the launcher has been dropped.
+ */
+static bool
+should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TimestampTz now;
+
+ Assert(TransactionIdIsValid(rdt_data->candidate_xid));
+ Assert(rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS ||
+ rdt_data->phase == RDT_WAIT_FOR_LOCAL_FLUSH);
+
+ if (!MySubscription->maxretention)
+ 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 = rdt_data->last_recv_time ? rdt_data->last_recv_time : GetCurrentTimestamp();
+
/*
- * Reset all data fields except those used to determine the timing for the
- * next round of transaction ID advancement. We can even use
- * flushpos_update_time in the next round to decide whether to get the
- * latest flush position.
+ * Return early if the wait time has not exceeded the configured maximum
+ * (max_retention_duration). Time spent waiting for table synchronization
+ * is excluded from this calculation, as it occurs infrequently.
*/
+ if (!TimestampDifferenceExceeds(rdt_data->candidate_xid_time, now,
+ MySubscription->maxretention +
+ rdt_data->table_sync_wait_time))
+ return false;
+
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+
+ return true;
+}
+
+/*
+ * Workhorse for the RDT_STOP_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Do not update the catalog during an active transaction. The transaction
+ * may be started during change application, leading to a possible
+ * rollback of catalog updates if the application fails subsequently.
+ */
+ if (IsTransactionState())
+ return;
+
+ StartTransactionCommand();
+
+ /*
+ * Updating pg_subscription might involve TOAST table access, so ensure we
+ * have a valid snapshot.
+ */
+ PushActiveSnapshot(GetTransactionSnapshot());
+
+ /* Set pg_subscription.subretentionactive to false */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+
+ PopActiveSnapshot();
+ CommitTransactionCommand();
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ /* Notify launcher to update the conflict slot */
+ ApplyLauncherWakeup();
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Reset all data fields of RetainDeadTuplesData except those used to
+ * determine the timing for the next round of transaction ID advancement. We
+ * can even use flushpos_update_time in the next round to decide whether to get
+ * the latest flush position.
+ */
+static void
+reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
+{
rdt_data->phase = RDT_GET_CANDIDATE_XID;
rdt_data->remote_lsn = InvalidXLogRecPtr;
rdt_data->remote_oldestxid = InvalidFullTransactionId;
@@ -4607,22 +4780,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
rdt_data->reply_time = 0;
rdt_data->remote_wait_for = InvalidFullTransactionId;
rdt_data->candidate_xid = InvalidTransactionId;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ rdt_data->table_sync_wait_time = 0;
}
/*
* 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.
+ * If there is no activity on the node, we progressively double the interval
+ * used to advance non-removable transaction ID. This helps conserve CPU
+ * and network resources when there's little benefit to frequent updates.
+ *
+ * The interval is capped by the lowest of the following:
+ * - wal_receiver_status_interval (if set),
+ * - a default maximum of 3 minutes,
+ * - max_retention_duration.
*
- * The interval is reset to a minimum value of 100ms once there is some
- * activity on the node.
+ * This ensures the interval never exceeds the retention boundary, even if
+ * other limits are higher. Once activity resumes on the node, the interval
+ * is reset to lesser of 100ms and max_retention_duration, allowing timely
+ * advancement of non-removable transaction ID.
*
* 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.
@@ -4651,6 +4827,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
*/
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
+
+ /* Ensure the wait time remains within the maximum limit */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
@@ -5458,11 +5638,12 @@ InitializeLogRepWorker(void)
* dropped, a restart is initiated.
*
* The oldest_nonremovable_xid should be initialized only when the
- * retain_dead_tuples is enabled before launching the worker. See
+ * subscription's retention is active before launching the worker. See
* logicalrep_worker_launch.
*/
if (am_leader_apply_worker() &&
MySubscription->retaindeadtuples &&
+ MySubscription->retentionactive &&
!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid))
{
ereport(LOG,
@@ -5633,8 +5814,9 @@ DisableSubscriptionAndExit(void)
* an error, as verifying commit timestamps is unnecessary in this
* context.
*/
- if (MySubscription->retaindeadtuples)
- CheckSubDeadTupleRetention(false, true, WARNING);
+ CheckSubDeadTupleRetention(false, true, WARNING,
+ MySubscription->retaindeadtuples,
+ MySubscription->retentionactive, false);
proc_exit(0);
}
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index fc7a6639163..bea793456f9 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -5048,6 +5048,7 @@ getSubscriptions(Archive *fout)
int i_subenabled;
int i_subfailover;
int i_subretaindeadtuples;
+ int i_submaxretention;
int i,
ntups;
@@ -5127,10 +5128,17 @@ getSubscriptions(Archive *fout)
if (fout->remoteVersion >= 190000)
appendPQExpBufferStr(query,
- " s.subretaindeadtuples\n");
+ " s.subretaindeadtuples,\n");
else
appendPQExpBufferStr(query,
- " false AS subretaindeadtuples\n");
+ " false AS subretaindeadtuples,\n");
+
+ if (fout->remoteVersion >= 190000)
+ appendPQExpBufferStr(query,
+ " s.submaxretention\n");
+ else
+ appendPQExpBuffer(query,
+ " 0 AS submaxretention\n");
appendPQExpBufferStr(query,
"FROM pg_subscription s\n");
@@ -5165,6 +5173,7 @@ getSubscriptions(Archive *fout)
i_subrunasowner = PQfnumber(res, "subrunasowner");
i_subfailover = PQfnumber(res, "subfailover");
i_subretaindeadtuples = PQfnumber(res, "subretaindeadtuples");
+ i_submaxretention = PQfnumber(res, "submaxretention");
i_subconninfo = PQfnumber(res, "subconninfo");
i_subslotname = PQfnumber(res, "subslotname");
i_subsynccommit = PQfnumber(res, "subsynccommit");
@@ -5200,6 +5209,8 @@ getSubscriptions(Archive *fout)
(strcmp(PQgetvalue(res, i, i_subfailover), "t") == 0);
subinfo[i].subretaindeadtuples =
(strcmp(PQgetvalue(res, i, i_subretaindeadtuples), "t") == 0);
+ subinfo[i].submaxretention =
+ atoi(PQgetvalue(res, i, i_submaxretention));
subinfo[i].subconninfo =
pg_strdup(PQgetvalue(res, i, i_subconninfo));
if (PQgetisnull(res, i, i_subslotname))
@@ -5461,6 +5472,9 @@ dumpSubscription(Archive *fout, const SubscriptionInfo *subinfo)
if (subinfo->subretaindeadtuples)
appendPQExpBufferStr(query, ", retain_dead_tuples = true");
+ if (subinfo->submaxretention)
+ appendPQExpBuffer(query, ", max_retention_duration = %d", subinfo->submaxretention);
+
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 dde85ed156c..bcc94ff07cc 100644
--- a/src/bin/pg_dump/pg_dump.h
+++ b/src/bin/pg_dump/pg_dump.h
@@ -717,6 +717,7 @@ typedef struct _SubscriptionInfo
bool subrunasowner;
bool subfailover;
bool subretaindeadtuples;
+ int submaxretention;
char *subconninfo;
char *subslotname;
char *subsynccommit;
diff --git a/src/bin/psql/describe.c b/src/bin/psql/describe.c
index 7a06af48842..4aa793d7de7 100644
--- a/src/bin/psql/describe.c
+++ b/src/bin/psql/describe.c
@@ -6746,7 +6746,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, false, false, false};
if (pset.sversion < 100000)
{
@@ -6815,10 +6815,20 @@ describeSubscriptions(const char *pattern, bool verbose)
", subfailover AS \"%s\"\n",
gettext_noop("Failover"));
if (pset.sversion >= 190000)
+ {
appendPQExpBuffer(&buf,
", subretaindeadtuples AS \"%s\"\n",
gettext_noop("Retain dead tuples"));
+ appendPQExpBuffer(&buf,
+ ", submaxretention AS \"%s\"\n",
+ gettext_noop("Max retention duration"));
+
+ appendPQExpBuffer(&buf,
+ ", subretentionactive AS \"%s\"\n",
+ gettext_noop("Retention active"));
+ }
+
appendPQExpBuffer(&buf,
", subsynccommit AS \"%s\"\n"
", subconninfo AS \"%s\"\n",
diff --git a/src/bin/psql/tab-complete.in.c b/src/bin/psql/tab-complete.in.c
index 8b10f2313f3..6b20a4404b2 100644
--- a/src/bin/psql/tab-complete.in.c
+++ b/src/bin/psql/tab-complete.in.c
@@ -2321,7 +2321,8 @@ match_previous_words(int pattern_id,
COMPLETE_WITH("(", "PUBLICATION");
/* ALTER SUBSCRIPTION <name> SET ( */
else if (Matches("ALTER", "SUBSCRIPTION", MatchAny, MatchAnyN, "SET", "("))
- COMPLETE_WITH("binary", "disable_on_error", "failover", "origin",
+ COMPLETE_WITH("binary", "disable_on_error", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
@@ -3780,7 +3781,8 @@ match_previous_words(int pattern_id,
/* Complete "CREATE SUBSCRIPTION <name> ... WITH ( <opt>" */
else if (Matches("CREATE", "SUBSCRIPTION", MatchAnyN, "WITH", "("))
COMPLETE_WITH("binary", "connect", "copy_data", "create_slot",
- "disable_on_error", "enabled", "failover", "origin",
+ "disable_on_error", "enabled", "failover",
+ "max_retention_duration", "origin",
"password_required", "retain_dead_tuples",
"run_as_owner", "slot_name", "streaming",
"synchronous_commit", "two_phase");
diff --git a/src/include/catalog/pg_subscription.h b/src/include/catalog/pg_subscription.h
index 231ef84ec9a..55cb9b1eefa 100644
--- a/src/include/catalog/pg_subscription.h
+++ b/src/include/catalog/pg_subscription.h
@@ -81,6 +81,15 @@ CATALOG(pg_subscription,6100,SubscriptionRelationId) BKI_SHARED_RELATION BKI_ROW
bool subretaindeadtuples; /* True if dead tuples useful for
* conflict detection are retained */
+ int32 submaxretention; /* The maximum duration (in milliseconds)
+ * for which information useful for
+ * conflict detection can be retained */
+
+ bool subretentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
+
#ifdef CATALOG_VARLEN /* variable-length fields start here */
/* Connection string to the publisher */
text subconninfo BKI_FORCE_NOT_NULL;
@@ -136,6 +145,13 @@ typedef struct Subscription
* to be synchronized to the standbys. */
bool retaindeadtuples; /* True if dead tuples useful for conflict
* detection are retained */
+ int32 maxretention; /* The maximum duration (in milliseconds) for
+ * which information useful for conflict
+ * detection can be retained */
+ bool retentionactive; /* True if retain_dead_tuples is enabled
+ * and the retention duration has not
+ * exceeded max_retention_duration, when
+ * defined */
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/include/catalog/pg_subscription_rel.h b/src/include/catalog/pg_subscription_rel.h
index f458447a0e5..02f97a547dd 100644
--- a/src/include/catalog/pg_subscription_rel.h
+++ b/src/include/catalog/pg_subscription_rel.h
@@ -92,4 +92,6 @@ extern void RemoveSubscriptionRel(Oid subid, Oid relid);
extern bool HasSubscriptionRelations(Oid subid);
extern List *GetSubscriptionRelations(Oid subid, bool not_ready);
+extern void UpdateDeadTupleRetentionStatus(Oid subid, bool active);
+
#endif /* PG_SUBSCRIPTION_REL_H */
diff --git a/src/include/commands/subscriptioncmds.h b/src/include/commands/subscriptioncmds.h
index 9b288ad22a6..fb4e26a51a4 100644
--- a/src/include/commands/subscriptioncmds.h
+++ b/src/include/commands/subscriptioncmds.h
@@ -31,6 +31,9 @@ extern char defGetStreamingMode(DefElem *def);
extern ObjectAddress AlterSubscription(ParseState *pstate, AlterSubscriptionStmt *stmt, bool isTopLevel);
extern void CheckSubDeadTupleRetention(bool check_guc, bool sub_disabled,
- int elevel_for_sub_disabled);
+ int elevel_for_sub_disabled,
+ bool retain_dead_tuples,
+ bool retention_active,
+ bool max_retention_set);
#endif /* SUBSCRIPTIONCMDS_H */
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 7c0204dd6f4..62ea1a00580 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -94,6 +94,9 @@ typedef struct LogicalRepWorker
* 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.
+ *
+ * This ID is set to InvalidTransactionId when the apply worker stops
+ * retaining information needed for conflict detection.
*/
TransactionId oldest_nonremovable_xid;
diff --git a/src/test/regress/expected/subscription.out b/src/test/regress/expected/subscription.out
index a98c97f7616..c7f1266fc2f 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | none | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+------------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub4 | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | f | t | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00012345
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00012345
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+---------------------+---------------------------+---------+---------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+------------------------------+------------
+ regress_testsub_foo | regress_subscription_user | f | {testpub2,testpub3} | f | parallel | d | f | any | t | f | f | f | 0 | f | local | dbname=regress_doesnotexist2 | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | t | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-----------------------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub,testpub1,testpub2} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | off | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | on | p | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -409,18 +409,18 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | t | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(1 row)
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
@@ -433,10 +433,36 @@ 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 | Retain dead tuples | 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/00000000
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
+DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+ERROR: max_retention_duration requires an integer value
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+NOTICE: max_retention_duration is ineffective when retain_dead_tuples 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 dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 1000 | f | off | dbname=regress_doesnotexist | 0/00000000
+(1 row)
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+\dRs+
+ List of subscriptions
+ Name | Owner | Enabled | Publication | Binary | Streaming | Two-phase commit | Disable on error | Origin | Password required | Run as owner? | Failover | Retain dead tuples | Max retention duration | Retention active | Synchronous commit | Conninfo | Skip LSN
+-----------------+---------------------------+---------+-------------+--------+-----------+------------------+------------------+--------+-------------------+---------------+----------+--------------------+------------------------+------------------+--------------------+-----------------------------+------------
+ regress_testsub | regress_subscription_user | f | {testpub} | f | parallel | d | f | any | t | f | f | f | 0 | f | off | dbname=regress_doesnotexist | 0/00000000
(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 f0f714fe747..ef0c298d2df 100644
--- a/src/test/regress/sql/subscription.sql
+++ b/src/test/regress/sql/subscription.sql
@@ -298,6 +298,22 @@ CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUB
ALTER SUBSCRIPTION regress_testsub SET (slot_name = NONE);
DROP SUBSCRIPTION regress_testsub;
+-- fail - max_retention_duration must be integer
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = foo);
+
+-- ok
+CREATE SUBSCRIPTION regress_testsub CONNECTION 'dbname=regress_doesnotexist' PUBLICATION testpub WITH (connect = false, max_retention_duration = 1000);
+
+\dRs+
+
+-- ok
+ALTER SUBSCRIPTION regress_testsub SET (max_retention_duration = 0);
+
+\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;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 6b4a9fb8815..51b23a39fa9 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -386,6 +386,59 @@ ok( $logfile =~
.*Remote row \(2, 4\); replica identity full \(2, 2\)/,
'update target row was deleted in tab');
+###############################################################################
+# Check that dead tuple retention stops due to the wait time surpassing
+# max_retention_duration.
+###############################################################################
+
+# Create a physical slot
+$node_B->safe_psql('postgres',
+ "SELECT * FROM pg_create_physical_replication_slot('blocker');");
+
+# Add the inactive physical slot to synchronized_standby_slots
+$node_B->append_conf('postgresql.conf',
+ "synchronized_standby_slots = 'blocker'");
+$node_B->reload;
+
+# Enable failover to activate the synchronized_standby_slots setting
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB DISABLE;");
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB SET (failover = true);");
+$node_A->safe_psql('postgres', "ALTER SUBSCRIPTION $subname_AB ENABLE;");
+
+# Insert a record
+$node_B->safe_psql('postgres', "INSERT INTO tab VALUES (5, 5);");
+
+# Advance the xid on Node A to trigger the next cycle of oldest_nonremovable_xid
+# advancement.
+$node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+$log_offset = -s $node_A->logfile;
+
+# Set max_retention_duration to a minimal value to initiate retention stop.
+$node_A->safe_psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 1);");
+
+# Confirm that the retention is stopped
+$node_A->wait_for_log(
+ qr/logical replication worker for subscription "tap_sub_a_b" has stopped retaining the information for detecting conflicts/,
+ $log_offset);
+
+ok( $node_A->poll_query_until(
+ 'postgres',
+ "SELECT xmin IS NULL from pg_replication_slots WHERE slot_name = 'pg_conflict_detection'"
+ ),
+ "the xmin value of slot 'pg_conflict_detection' is invalid on Node A");
+
+$result = $node_A->safe_psql('postgres',
+ "SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
+is($result, qq(f), 'retention is inactive');
+
+# Drop the physical slot and reset the synchronized_standby_slots setting
+$node_B->safe_psql('postgres',
+ "SELECT * FROM pg_drop_replication_slot('blocker');");
+$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
+$node_B->reload;
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
v70-0002-Resume-retaining-the-information-for-conflict-de.patchapplication/octet-stream; name=v70-0002-Resume-retaining-the-information-for-conflict-de.patchDownload
From 44162d3da40b3ba7f295da79efe8ab5507e632e1 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 1 Sep 2025 12:40:36 +0800
Subject: [PATCH v70 2/3] Resume retaining the information for conflict
detection
The patch allows the launcher to re-initialized invalidated slot, if at
least one apply worker has confirmed that the retention duration is now within
the max_retention_duration.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/commands/subscriptioncmds.c | 9 +-
src/backend/replication/logical/launcher.c | 48 ++++-
src/backend/replication/logical/worker.c | 206 ++++++++++++++++++---
src/include/replication/worker_internal.h | 6 +
5 files changed, 240 insertions(+), 38 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ee22d0fb3ca 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will be automatically resumed once at least
+ one apply worker confirms that the retention duration is within the
+ specified limit, or a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>, or the user manually
+ re-enables <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/commands/subscriptioncmds.c b/src/backend/commands/subscriptioncmds.c
index 82cf65fae73..cc622fce58f 100644
--- a/src/backend/commands/subscriptioncmds.c
+++ b/src/backend/commands/subscriptioncmds.c
@@ -854,7 +854,14 @@ CreateSubscription(ParseState *pstate, CreateSubscriptionStmt *stmt,
pgstat_create_subscription(subid);
- if (opts.enabled)
+ /*
+ * If the subscription is enabled, notify the launcher to start the apply
+ * worker.
+ *
+ * If the subscription has retain_dead_tuples enabled, notify the launcher
+ * to create or resume the conflict detection slot.
+ */
+ if (opts.enabled || opts.retaindeadtuples)
ApplyLauncherWakeupAtCommit();
ObjectAddressSet(myself, SubscriptionRelationId, subid);
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add2e2e066c..226421e3dfa 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,7 +101,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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
static void init_conflict_slot_xmin(void);
@@ -468,6 +470,7 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->wait_for_initial_xid = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1270,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1382,11 +1383,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * Additionally, if an apply worker has an invalid XID and is requesting to
+ * resume retention, assign the slot's xmin value to it.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool wait_for_xid;
Assert(worker != NULL);
@@ -1398,16 +1404,42 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ wait_for_xid = worker->wait_for_initial_xid;
SpinLockRelease(&worker->relmutex);
/*
- * Return if the apply worker has stopped retention concurrently.
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if
+ * requested. This ensures the apply worker continues to maintain the
+ * oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (wait_for_xid)
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return if the apply worker has stopped retention concurrently and has not
+ * yet resumed.
*
* Although this function is invoked only when retentionactive is true,
* the apply worker might stop retention after the launcher fetches the
* retentionactive flag.
*/
- if (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ if (!can_update_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index f5f9183bdaf..0f014f44db4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION
+ * at any phase if the retention has been stopped, but max_retention_duration
+ * is now set to 0.
+ *
* 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) ->
@@ -382,7 +395,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -569,6 +583,10 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4339,6 +4357,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4361,10 +4386,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4393,6 +4414,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4649,6 +4673,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4679,9 +4715,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
*
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4712,10 +4747,16 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
return true;
}
@@ -4725,6 +4766,131 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !MySubscription->retentionactive;
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = false;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4742,26 +4908,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index 62ea1a00580..58f2dcc7f0b 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -100,6 +100,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker is resuming retention and is waiting
+ * for the launcher to initialize oldest_nonremovable_xid.
+ */
+ bool wait_for_initial_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
--
2.51.0.windows.1
On Mon, Sep 1, 2025 at 5:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I reviewed the patch internally and tweaked a small detail of the apply worker
to reduce the waiting time in the main loop when max_retention_duration is
defined (set wait_time = min(wait_time, max_retention_duration)). Also, I added
a simple test in 035_conflicts.pl of 0001 to verify the new sub option.Here is V70 patch set.
The patch v70-0001 looks good to me. Verified, all the old issues are resolved.
Will resume review of v70-0002 now.
thanks
Shveta
On Mon, Sep 1, 2025 at 5:45 PM shveta malik <shveta.malik@gmail.com> wrote:
Here is V70 patch set.
The patch v70-0001 looks good to me. Verified, all the old issues are resolved.
Will resume review of v70-0002 now.
Please find a few comments on v70-0002:
1)
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * The retention will resume automatically if the worker has confirmed that the
+ * retention duration is now within the max_retention_duration.
Do we need this comment atop stop as it does not directly concern
stop? Isn't the details regarding RDT_RESUME_CONFLICT_INFO_RETENTION
in the file-header section sufficient?
2)
+ /* Stop retention if not yet */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+ }
+
+ reset_retention_data_fields(rdt_data);
should_stop_conflict_info_retention() does reset_retention_data_fields
everytime when wait-time exceeds the limit, and when it actually stops
i.e. calls stop_conflict_info_retention through phase change; the stop
function also does reset_retention_data_fields and calls
process_rdt_phase_transition. Can we optimize this code part by
consolidating the reset_retention_data_fields() and
process_rdt_phase_transition() calls into
should_stop_conflict_info_retention() itself, eliminating redundancy?
3)
Shall we update 035_conflicts.pl to have a resume test by setting
max_retention_duration to 0 after stop-retention test?
4)
+ subscription. The retention will be automatically resumed
once at least
+ one apply worker confirms that the retention duration is within the
+ specified limit, or a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>, or the user manually
+ re-enables <literal>retain_dead_tuples</literal>.
Shall we rephrase it slightly to:
Retention will automatically resume when at least one apply worker
confirms that the retention duration is within the specified limit, or
when a new subscription is created with <literal>retain_dead_tuples =
true</literal>. Alternatively, retention can be manually
resumed by re-enabling <literal>retain_dead_tuples</literal>.
thanks
Shveta
On Tue, Sep 2, 2025 at 3:30 PM shveta malik <shveta.malik@gmail.com> wrote:
Here is V70 patch set.
Please find a few comments on v70-003:
1)
Doc of dead_tuple_retention_active says:
True if retain_dead_tuples is enabled and the retention duration for
information used in conflict detection is within
max_retention_duration
Doc of subretentionactive says:
The retention status of information (e.g., dead tuples, commit
timestamps, and origins) useful for conflict detection. True if
retain_dead_tuples is enabled, and the retention duration has not
exceeded max_retention_duration, when defined.
There is hardly any difference between the two. Do we really need to
have 'dead_tuple_retention_active' when we already have
'subretentionactive'?
2)
Doc wise, there is no difference between the two, but there is a small
window when sub's subretentionactive will show true while stat's
dead_tuple_retention_active will show false. This will be when worker
is waiting for the launcher to assign its oldest-xid after it has
marked itself as 'resuming'.
If we decide to retain 'dead_tuple_retention_active', then do we need
to indicate the small difference between the 2 fields in the doc?
3)
We can add a test when we stop-retention to see if this is showing
false. Currently there are 2 places in the test where we check this
field to see if it is true. I think we can shift both in the same
test. One check before stop-retention, one check after stop-retention.
thanks
Shveta
Hi,
As reported by Robert[1]/messages/by-id/CA+TgmoaQtB=cnMJwCA33bDrGt7x5ysoW770uJ2Z56AU_NVNdbw@mail.gmail.com, it is worth adding a test for the race condition in
the RecordTransactionCommitPrepared() function to reduce the risk of future code
changes:
/*
* 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 transactions that have acquired commit timestamp
* 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 comments atop worker.c.
*/
committs = GetCurrentTimestamp();
While writing the test, I noticed a bug that conflict-relevant data could be
prematurely removed before applying prepared transactions on the publisher that
are in the commit phase. This occurred because GetOldestActiveTransactionId()
was called on the publisher, which failed to account for the backend executing
COMMIT PREPARED, as this backend does not have an xid stored in PGPROC.
Since this issue overlaps with the race condition related to timestamp
acquisition, I've prepared a bug fix along with a test for the race condition.
The 0001 patch fixes this issue by introducing a new function that iterates over
global transactions and identifies prepared transactions during the commit
phase. 0002 added injection points and tap-test to test the bug and timestamp
acquisition.
[1]: /messages/by-id/CA+TgmoaQtB=cnMJwCA33bDrGt7x5ysoW770uJ2Z56AU_NVNdbw@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v1-0002-Add-a-race-condition-test.patchapplication/octet-stream; name=v1-0002-Add-a-race-condition-test.patchDownload
From 919824c32531bfc5a80406b9d28019bb5ba0fef7 Mon Sep 17 00:00:00 2001
From: Nisha Moond <nisha.moond412@gmail.com>
Date: Tue, 27 May 2025 17:05:02 +0530
Subject: [PATCH v1 2/2] Add a race condition test
---
src/backend/access/transam/twophase.c | 6 +
src/test/subscription/Makefile | 4 +-
src/test/subscription/meson.build | 6 +-
.../t/036_confl_after_delay_chkpt.pl | 183 ++++++++++++++++++
4 files changed, 197 insertions(+), 2 deletions(-)
create mode 100644 src/test/subscription/t/036_confl_after_delay_chkpt.pl
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index e14e7551129..c96c5c09654 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -103,6 +103,7 @@
#include "storage/proc.h"
#include "storage/procarray.h"
#include "utils/builtins.h"
+#include "utils/injection_point.h"
#include "utils/memutils.h"
#include "utils/timestamp.h"
@@ -2332,12 +2333,17 @@ RecordTransactionCommitPrepared(TransactionId xid,
replorigin = (replorigin_session_origin != InvalidRepOriginId &&
replorigin_session_origin != DoNotReplicateId);
+ /* Load the injection point before entering the critical section */
+ INJECTION_POINT_LOAD("commit-after-delay-checkpoint");
+
START_CRIT_SECTION();
/* See notes in RecordTransactionCommit */
Assert((MyProc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0);
MyProc->delayChkptFlags |= DELAY_CHKPT_IN_COMMIT;
+ INJECTION_POINT_CACHED("commit-after-delay-checkpoint", NULL);
+
/*
* Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
* commit time is written.
diff --git a/src/test/subscription/Makefile b/src/test/subscription/Makefile
index 50b65d8f6ea..9d97e7d5c0d 100644
--- a/src/test/subscription/Makefile
+++ b/src/test/subscription/Makefile
@@ -13,9 +13,11 @@ subdir = src/test/subscription
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-EXTRA_INSTALL = contrib/hstore
+EXTRA_INSTALL = contrib/hstore \
+ src/test/modules/injection_points
export with_icu
+export enable_injection_points
check:
$(prove_check)
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index 586ffba434e..65b8493eedc 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -5,7 +5,10 @@ tests += {
'sd': meson.current_source_dir(),
'bd': meson.current_build_dir(),
'tap': {
- 'env': {'with_icu': icu.found() ? 'yes' : 'no'},
+ 'env': {
+ 'with_icu': icu.found() ? 'yes' : 'no',
+ 'enable_injection_points': get_option('injection_points') ? 'yes' : 'no',
+ },
'tests': [
't/001_rep_changes.pl',
't/002_types.pl',
@@ -42,6 +45,7 @@ tests += {
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
't/035_conflicts.pl',
+ 't/036_confl_after_delay_chkpt.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/036_confl_after_delay_chkpt.pl b/src/test/subscription/t/036_confl_after_delay_chkpt.pl
new file mode 100644
index 00000000000..f57ee5eea4d
--- /dev/null
+++ b/src/test/subscription/t/036_confl_after_delay_chkpt.pl
@@ -0,0 +1,183 @@
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test that publisher's transactions marked with DELAY_CHKPT_IN_COMMIT prevent
+# concurrently deleted tuples on the subscriber from being removed.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+# This test depends on an injection point to block the transaction commit after
+# marking DELAY_CHKPT_IN_COMMIT flag.
+if ($ENV{enable_injection_points} ne 'yes')
+{
+ plan skip_all => 'Injection points not supported by this build';
+}
+
+# Create a publisher node. Disable autovacuum to stablized the tests related to
+# manual VACUUM and transaction ID.
+my $node_publisher = PostgreSQL::Test::Cluster->new('publisher');
+$node_publisher->init(allows_streaming => 'logical');
+$node_publisher->append_conf(
+ 'postgresql.conf', qq{
+autovacuum = off
+track_commit_timestamp = on
+max_prepared_transactions = 1
+shared_preload_libraries = 'injection_points'
+});
+$node_publisher->start;
+
+# Check if the 'injection_points' extension is available, as it may be
+# possible that this script is run with installcheck, where the module
+# would not be installed by default.
+if (!$node_publisher->check_extension('injection_points'))
+{
+ plan skip_all => 'Extension injection_points not installed';
+}
+
+my $node_publisher_connstr = $node_publisher->connstr . ' dbname=postgres';
+
+# Create a subscriber node
+my $node_subscriber = PostgreSQL::Test::Cluster->new('subscriber');
+$node_subscriber->init(allows_streaming => 'logical');
+$node_subscriber->append_conf(
+ 'postgresql.conf', qq{
+autovacuum = off
+track_commit_timestamp = on
+});
+$node_subscriber->start;
+
+# Create a table and publication on publisher
+$node_publisher->safe_psql('postgres',
+ "CREATE TABLE tab (a int PRIMARY KEY, b int)");
+$node_publisher->safe_psql('postgres',
+ "CREATE PUBLICATION tab_pub FOR TABLE tab");
+
+# Insert some data
+$node_publisher->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1);");
+
+# Create the same table on subscriber and create a subscription
+my $subname = 'tab_sub';
+$node_subscriber->safe_psql('postgres',
+ "CREATE TABLE tab (a int PRIMARY KEY, b int)");
+$node_subscriber->safe_psql(
+ 'postgres', "
+ CREATE SUBSCRIPTION $subname
+ CONNECTION '$node_publisher_connstr application_name=$subname'
+ PUBLICATION tab_pub
+ WITH (retain_dead_tuples = true)");
+
+# Wait for initial table sync to finish
+$node_subscriber->wait_for_subscription_sync($node_publisher, $subname);
+
+ok( $node_subscriber->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 subscriber");
+
+# Create the injection_points extension on the publisher node and attach to the
+# commit-after-delay-checkpoint injection point.
+$node_publisher->safe_psql(
+ 'postgres',
+ "CREATE EXTENSION injection_points;
+ SELECT injection_points_attach('commit-after-delay-checkpoint', 'wait');"
+);
+
+# Start a background session on the publisher node to perform an update and
+# pause at the injection point.
+my $pub_session = $node_publisher->background_psql('postgres');
+$pub_session->query_until(
+ qr/starting_bg_psql/,
+ q{
+ \echo starting_bg_psql
+ BEGIN;
+ UPDATE tab SET b = 2 WHERE a = 1;
+ PREPARE TRANSACTION 'txn_with_later_commit_ts';
+ COMMIT PREPARED 'txn_with_later_commit_ts';
+ }
+);
+
+# Confirm the update is suspended
+my $result =
+ $node_publisher->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
+is($result, qq(1|1), 'publisher sees the old row');
+
+# Delete the row on the subscriber. The deleted row should be retained due to a
+# transaction on the publisher, which is currently marked with the
+# DELAY_CHKPT_IN_COMMIT flag.
+$node_subscriber->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+# Get the commit timestamp for the delete
+my $sub_ts = $node_subscriber->safe_psql('postgres',
+ "SELECT timestamp FROM pg_last_committed_xact();");
+
+# Confirm that the dead tuple can be removed now
+my ($cmdret, $stdout, $stderr) =
+ $node_subscriber->psql('postgres', qq(VACUUM (verbose) public.tab;));
+
+ok($stderr =~ qr/1 are dead but not yet removable/,
+ 'the deleted column is non-removable');
+
+my $log_location = -s $node_subscriber->logfile;
+
+# Wakeup and detach the injection point on the publisher node. The prepared
+# transaction should now commit.
+$node_publisher->safe_psql(
+ 'postgres',
+ "SELECT injection_points_wakeup('commit-after-delay-checkpoint');
+ SELECT injection_points_detach('commit-after-delay-checkpoint');"
+);
+
+# Close the background session on the publisher node
+ok($pub_session->quit, "close publisher session");
+
+# Confirm that the transaction committed
+$result =
+ $node_publisher->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
+is($result, qq(1|2), 'publisher sees the new row');
+
+# Ensure the UPDATE is replayed on subscriber
+$node_publisher->wait_for_catchup($subname);
+
+my $logfile = slurp_file($node_subscriber->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 row \(1, 2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+# Remember the next transaction ID to be assigned
+my $next_xid =
+ $node_subscriber->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+# Confirm that the xmin value is advanced to the latest nextXid even if there is
+# one transaction on the publisher that has not committed.
+ok( $node_subscriber->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 subscriber"
+);
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) =
+ $node_subscriber->psql('postgres', qq(VACUUM (verbose) public.tab;));
+
+ok($stderr =~ qr/1 removed, 0 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+# Get the commit timestamp for the publisher's update
+my $pub_ts = $node_publisher->safe_psql('postgres',
+ "SELECT pg_xact_commit_timestamp(xmin) from tab where a=1;");
+
+# Check that the commit timestamp for the update on the publisher is later than
+# or equal to the timestamp of the local deletion, as the commit timestamp
+# should be assigned after marking the DELAY_CHKPT_IN_COMMIT flag.
+$result = $node_publisher->safe_psql('postgres',
+ "SELECT '$pub_ts'::timestamp >= '$sub_ts'::timestamp");
+is($result, qq(t),
+ "pub UPDATE's timestamp is later than that of sub's DELETE");
+
+done_testing();
--
2.51.0.windows.1
v1-0001-Fix-conflict-relevant-data-retention-for-prepared.patchapplication/octet-stream; name=v1-0001-Fix-conflict-relevant-data-retention-for-prepared.patchDownload
From 4dfb1607c255e351e0e60b3c4175c7299ec6dbf0 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Thu, 4 Sep 2025 12:50:59 +0800
Subject: [PATCH v1 1/2] Fix conflict-relevant data retention for prepared
transactions
Previously, conflict-relevant data could be prematurely removed before applying
prepared transactions on the publisher that are in the commit phase. This
occurred because GetOldestActiveTransactionId() was called on the publisher,
which failed to account for the backend executing COMMIT PREPARED, as this
backend does not have an xid stored in PGPROC.
This commit fixes the issue by introducing a new function to traverse global
transactions, identifying prepared transactions in the commit phase.
Additionally, an injection point has been added to ensure that data is not
prematurely removed when a concurrent prepared transaction is being committed on
the publisher.
---
src/backend/access/transam/twophase.c | 56 +++++++++++++++++++++++++++
src/backend/replication/walsender.c | 12 ++++++
src/include/access/twophase.h | 2 +
3 files changed, 70 insertions(+)
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 7918176fc58..e14e7551129 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2809,3 +2809,59 @@ LookupGXactBySubid(Oid subid)
return found;
}
+
+/*
+ * TwoPhaseGetXidByLockingProc
+ * Return the oldest transaction ID from prepared transactions that are
+ * currently in the commit phase.
+ *
+ * This function only considers transactions in the currently connected
+ * database. If no matching transactions are found, it returns
+ * InvalidTransactionId.
+ */
+TransactionId
+TwoPhaseGetOldestXidInCommit(void)
+{
+ TransactionId oldestRunningXid = InvalidTransactionId;
+
+ LWLockAcquire(TwoPhaseStateLock, LW_SHARED);
+
+ for (int i = 0; i < TwoPhaseState->numPrepXacts; i++)
+ {
+ GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+ PGPROC *proc = GetPGProcByNumber(gxact->pgprocno);
+ PGPROC *commitproc;
+ TransactionId xid;
+
+ if (!gxact->valid)
+ continue;
+
+ if (MyDatabaseId != proc->databaseId)
+ continue;
+
+ if (gxact->locking_backend == INVALID_PROC_NUMBER)
+ continue;
+
+ /*
+ * Get the backend that is handling the transaction. It's safe to
+ * access this backend while holding TwoPhaseStateLock, as the backend
+ * can only be destroyed after either removing or unlocking the
+ * current global transaction, both of which require an exclusive
+ * TwoPhaseStateLock.
+ */
+ commitproc = GetPGProcByNumber(gxact->locking_backend);
+
+ if ((commitproc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
+ xid = XidFromFullTransactionId(gxact->fxid);
+
+ if (!TransactionIdIsValid(oldestRunningXid) ||
+ TransactionIdPrecedes(xid, oldestRunningXid))
+ oldestRunningXid = xid;
+ }
+
+ LWLockRelease(TwoPhaseStateLock);
+
+ return oldestRunningXid;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index e3dce9dc68d..59822f22b8d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -51,6 +51,7 @@
#include "access/timeline.h"
#include "access/transam.h"
+#include "access/twophase.h"
#include "access/xact.h"
#include "access/xlog_internal.h"
#include "access/xlogreader.h"
@@ -2719,6 +2720,7 @@ ProcessStandbyPSRequestMessage(void)
{
XLogRecPtr lsn = InvalidXLogRecPtr;
TransactionId oldestXidInCommit;
+ TransactionId oldestGXidInCommit;
FullTransactionId nextFullXid;
FullTransactionId fullOldestXidInCommit;
WalSnd *walsnd = MyWalSnd;
@@ -2746,6 +2748,16 @@ ProcessStandbyPSRequestMessage(void)
* ones replicated.
*/
oldestXidInCommit = GetOldestActiveTransactionId(true, false);
+ oldestGXidInCommit = TwoPhaseGetOldestXidInCommit();
+
+ /*
+ * Update the oldest xid for standby transmission if an older prepared
+ * transaction exists and is currently in commit phase.
+ */
+ if (TransactionIdIsValid(oldestGXidInCommit) &&
+ TransactionIdPrecedes(oldestGXidInCommit, oldestXidInCommit))
+ oldestXidInCommit = oldestGXidInCommit;
+
nextFullXid = ReadNextFullTransactionId();
fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
oldestXidInCommit);
diff --git a/src/include/access/twophase.h b/src/include/access/twophase.h
index 509bdad9a5d..64463e9f4af 100644
--- a/src/include/access/twophase.h
+++ b/src/include/access/twophase.h
@@ -68,4 +68,6 @@ extern void TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid_res,
int szgid);
extern bool LookupGXactBySubid(Oid subid);
+extern TransactionId TwoPhaseGetOldestXidInCommit(void);
+
#endif /* TWOPHASE_H */
--
2.51.0.windows.1
On Thu, Sep 4, 2025 at 3:30 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Hi,
As reported by Robert[1], it is worth adding a test for the race condition in
the RecordTransactionCommitPrepared() function to reduce the risk of future code
changes:/*
* 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 transactions that have acquired commit timestamp
* 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 comments atop worker.c.
*/
committs = GetCurrentTimestamp();While writing the test, I noticed a bug that conflict-relevant data could be
prematurely removed before applying prepared transactions on the publisher that
are in the commit phase. This occurred because GetOldestActiveTransactionId()
was called on the publisher, which failed to account for the backend executing
COMMIT PREPARED, as this backend does not have an xid stored in PGPROC.Since this issue overlaps with the race condition related to timestamp
acquisition, I've prepared a bug fix along with a test for the race condition.
The 0001 patch fixes this issue by introducing a new function that iterates over
global transactions and identifies prepared transactions during the commit
phase. 0002 added injection points and tap-test to test the bug and timestamp
acquisition.
Thank You for the patches.Verified 001, works well. Just one minor comment:
1)
+ PGPROC *proc = GetPGProcByNumber(gxact->pgprocno);
+ PGPROC *commitproc;
We get first proc and then commitproc. proc is used only to get
databaseId, why don't we get databaseId from commitproc itself?
~~
Few trivial comments for 002:
1)
+# Test that publisher's transactions marked with DELAY_CHKPT_IN_COMMIT prevent
+# concurrently deleted tuples on the subscriber from being removed.
Here shall we also mention something like:
This test also acts as a safeguard to prevent developers from moving
the timestamp acquisition before marking DELAY_CHKPT_IN_COMMIT in
RecordTransactionCommitPrepared.
2)
# This test depends on an injection point to block the transaction commit after
# marking DELAY_CHKPT_IN_COMMIT flag.
Shall we say:
..to block the prepared transaction commit..
3)
+# Create a publisher node. Disable autovacuum to stablized the tests related to
+# manual VACUUM and transaction ID.
to stablized --> to stabilize
4)
+# Confirm that the dead tuple can be removed now
+my ($cmdret, $stdout, $stderr) =
+ $node_subscriber->psql('postgres', qq(VACUUM (verbose) public.tab;));
+
+ok($stderr =~ qr/1 are dead but not yet removable/,
+ 'the deleted column is non-removable');
can be removed now --> cannot be removed now
thanks
Shveta
On Friday, September 5, 2025 2:01 PM shveta malik <shveta.malik@gmail.com> wrote:
On Thu, Sep 4, 2025 at 3:30 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Hi,
As reported by Robert[1], it is worth adding a test for the race condition in
the RecordTransactionCommitPrepared() function to reduce the risk offuture code
changes:
/*
* Note it is important to set committs value after marking ourselvesas
* in the commit critical section (DELAY_CHKPT_IN_COMMIT).
This is because
* we want to ensure all transactions that have acquired commit
timestamp
* 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 comments atop worker.c.
*/
committs = GetCurrentTimestamp();While writing the test, I noticed a bug that conflict-relevant data could be
prematurely removed before applying prepared transactions on the publisherthat
are in the commit phase. This occurred because
GetOldestActiveTransactionId()
was called on the publisher, which failed to account for the backend
executing
COMMIT PREPARED, as this backend does not have an xid stored in
PGPROC.
Since this issue overlaps with the race condition related to timestamp
acquisition, I've prepared a bug fix along with a test for the race condition.
The 0001 patch fixes this issue by introducing a new function that iteratesover
global transactions and identifies prepared transactions during the commit
phase. 0002 added injection points and tap-test to test the bug andtimestamp
acquisition.
Thank You for the patches.Verified 001, works well. Just one minor comment:
1) + PGPROC *proc = GetPGProcByNumber(gxact->pgprocno); + PGPROC *commitproc; We get first proc and then commitproc. proc is used only to get databaseId, why don't we get databaseId from commitproc itself?
I think it's OK to directly refer to commitproc, so changed.
~~
Few trivial comments for 002:
Thanks for the comments.
1) +# Test that publisher's transactions marked with DELAY_CHKPT_IN_COMMIT prevent +# concurrently deleted tuples on the subscriber from being removed.Here shall we also mention something like:
This test also acts as a safeguard to prevent developers from moving
the timestamp acquisition before marking DELAY_CHKPT_IN_COMMIT in
RecordTransactionCommitPrepared.
Added.
2)
# This test depends on an injection point to block the transaction commit after
# marking DELAY_CHKPT_IN_COMMIT flag.Shall we say:
..to block the prepared transaction commit..
Changed.
3) +# Create a publisher node. Disable autovacuum to stablized the tests related to +# manual VACUUM and transaction ID.to stablized --> to stabilize
Fixed.
4) +# Confirm that the dead tuple can be removed now +my ($cmdret, $stdout, $stderr) = + $node_subscriber->psql('postgres', qq(VACUUM (verbose) public.tab;)); + +ok($stderr =~ qr/1 are dead but not yet removable/, + 'the deleted column is non-removable');can be removed now --> cannot be removed now
Fixed.
Here are v2 patches which addressed above comments.
Best Regards,
Hou zj
Attachments:
v2-0002-Add-a-race-condition-test.patchapplication/octet-stream; name=v2-0002-Add-a-race-condition-test.patchDownload
From bf2cfb8c410da60128aa7a3c082cd2bb393afa16 Mon Sep 17 00:00:00 2001
From: Nisha Moond <nisha.moond412@gmail.com>
Date: Tue, 27 May 2025 17:05:02 +0530
Subject: [PATCH v2 2/2] Add a race condition test
An injection point has been added to ensure that the conflict-relevant is not
prematurely removed when a concurrent prepared transaction is being committed on
the publisher.
---
src/backend/access/transam/twophase.c | 6 +
src/test/subscription/Makefile | 4 +-
src/test/subscription/meson.build | 6 +-
.../t/036_confl_after_delay_chkpt.pl | 186 ++++++++++++++++++
4 files changed, 200 insertions(+), 2 deletions(-)
create mode 100644 src/test/subscription/t/036_confl_after_delay_chkpt.pl
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 06a41ec70c8..39faf032b06 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -103,6 +103,7 @@
#include "storage/proc.h"
#include "storage/procarray.h"
#include "utils/builtins.h"
+#include "utils/injection_point.h"
#include "utils/memutils.h"
#include "utils/timestamp.h"
@@ -2332,12 +2333,17 @@ RecordTransactionCommitPrepared(TransactionId xid,
replorigin = (replorigin_session_origin != InvalidRepOriginId &&
replorigin_session_origin != DoNotReplicateId);
+ /* Load the injection point before entering the critical section */
+ INJECTION_POINT_LOAD("commit-after-delay-checkpoint");
+
START_CRIT_SECTION();
/* See notes in RecordTransactionCommit */
Assert((MyProc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0);
MyProc->delayChkptFlags |= DELAY_CHKPT_IN_COMMIT;
+ INJECTION_POINT_CACHED("commit-after-delay-checkpoint", NULL);
+
/*
* Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
* commit time is written.
diff --git a/src/test/subscription/Makefile b/src/test/subscription/Makefile
index 50b65d8f6ea..9d97e7d5c0d 100644
--- a/src/test/subscription/Makefile
+++ b/src/test/subscription/Makefile
@@ -13,9 +13,11 @@ subdir = src/test/subscription
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-EXTRA_INSTALL = contrib/hstore
+EXTRA_INSTALL = contrib/hstore \
+ src/test/modules/injection_points
export with_icu
+export enable_injection_points
check:
$(prove_check)
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index 586ffba434e..65b8493eedc 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -5,7 +5,10 @@ tests += {
'sd': meson.current_source_dir(),
'bd': meson.current_build_dir(),
'tap': {
- 'env': {'with_icu': icu.found() ? 'yes' : 'no'},
+ 'env': {
+ 'with_icu': icu.found() ? 'yes' : 'no',
+ 'enable_injection_points': get_option('injection_points') ? 'yes' : 'no',
+ },
'tests': [
't/001_rep_changes.pl',
't/002_types.pl',
@@ -42,6 +45,7 @@ tests += {
't/033_run_as_table_owner.pl',
't/034_temporal.pl',
't/035_conflicts.pl',
+ 't/036_confl_after_delay_chkpt.pl',
't/100_bugs.pl',
],
},
diff --git a/src/test/subscription/t/036_confl_after_delay_chkpt.pl b/src/test/subscription/t/036_confl_after_delay_chkpt.pl
new file mode 100644
index 00000000000..d9de2a64be3
--- /dev/null
+++ b/src/test/subscription/t/036_confl_after_delay_chkpt.pl
@@ -0,0 +1,186 @@
+# Copyright (c) 2025, PostgreSQL Global Development Group
+
+# Test that publisher's transactions marked with DELAY_CHKPT_IN_COMMIT prevent
+# concurrently deleted tuples on the subscriber from being removed. This test
+# also acts as a safeguard to prevent developers from moving the commit
+# timestamp acquisition before marking DELAY_CHKPT_IN_COMMIT in
+# RecordTransactionCommitPrepared.
+use strict;
+use warnings FATAL => 'all';
+use PostgreSQL::Test::Cluster;
+use PostgreSQL::Test::Utils;
+use Test::More;
+
+# This test depends on an injection point to block the prepared transaction
+# commit after marking DELAY_CHKPT_IN_COMMIT flag.
+if ($ENV{enable_injection_points} ne 'yes')
+{
+ plan skip_all => 'Injection points not supported by this build';
+}
+
+# Create a publisher node. Disable autovacuum to stabilize the tests related to
+# manual VACUUM and transaction ID.
+my $node_publisher = PostgreSQL::Test::Cluster->new('publisher');
+$node_publisher->init(allows_streaming => 'logical');
+$node_publisher->append_conf(
+ 'postgresql.conf', qq{
+autovacuum = off
+track_commit_timestamp = on
+max_prepared_transactions = 1
+shared_preload_libraries = 'injection_points'
+});
+$node_publisher->start;
+
+# Check if the 'injection_points' extension is available, as it may be
+# possible that this script is run with installcheck, where the module
+# would not be installed by default.
+if (!$node_publisher->check_extension('injection_points'))
+{
+ plan skip_all => 'Extension injection_points not installed';
+}
+
+my $node_publisher_connstr = $node_publisher->connstr . ' dbname=postgres';
+
+# Create a subscriber node
+my $node_subscriber = PostgreSQL::Test::Cluster->new('subscriber');
+$node_subscriber->init(allows_streaming => 'logical');
+$node_subscriber->append_conf(
+ 'postgresql.conf', qq{
+autovacuum = off
+track_commit_timestamp = on
+});
+$node_subscriber->start;
+
+# Create a table and publication on publisher
+$node_publisher->safe_psql('postgres',
+ "CREATE TABLE tab (a int PRIMARY KEY, b int)");
+$node_publisher->safe_psql('postgres',
+ "CREATE PUBLICATION tab_pub FOR TABLE tab");
+
+# Insert some data
+$node_publisher->safe_psql('postgres', "INSERT INTO tab VALUES (1, 1);");
+
+# Create the same table on subscriber and create a subscription
+my $subname = 'tab_sub';
+$node_subscriber->safe_psql('postgres',
+ "CREATE TABLE tab (a int PRIMARY KEY, b int)");
+$node_subscriber->safe_psql(
+ 'postgres', "
+ CREATE SUBSCRIPTION $subname
+ CONNECTION '$node_publisher_connstr application_name=$subname'
+ PUBLICATION tab_pub
+ WITH (retain_dead_tuples = true)");
+
+# Wait for initial table sync to finish
+$node_subscriber->wait_for_subscription_sync($node_publisher, $subname);
+
+ok( $node_subscriber->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 subscriber");
+
+# Create the injection_points extension on the publisher node and attach to the
+# commit-after-delay-checkpoint injection point.
+$node_publisher->safe_psql(
+ 'postgres',
+ "CREATE EXTENSION injection_points;
+ SELECT injection_points_attach('commit-after-delay-checkpoint', 'wait');"
+);
+
+# Start a background session on the publisher node to perform an update and
+# pause at the injection point.
+my $pub_session = $node_publisher->background_psql('postgres');
+$pub_session->query_until(
+ qr/starting_bg_psql/,
+ q{
+ \echo starting_bg_psql
+ BEGIN;
+ UPDATE tab SET b = 2 WHERE a = 1;
+ PREPARE TRANSACTION 'txn_with_later_commit_ts';
+ COMMIT PREPARED 'txn_with_later_commit_ts';
+ }
+);
+
+# Confirm the update is suspended
+my $result =
+ $node_publisher->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
+is($result, qq(1|1), 'publisher sees the old row');
+
+# Delete the row on the subscriber. The deleted row should be retained due to a
+# transaction on the publisher, which is currently marked with the
+# DELAY_CHKPT_IN_COMMIT flag.
+$node_subscriber->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+# Get the commit timestamp for the delete
+my $sub_ts = $node_subscriber->safe_psql('postgres',
+ "SELECT timestamp FROM pg_last_committed_xact();");
+
+# Confirm that the dead tuple cannot be removed
+my ($cmdret, $stdout, $stderr) =
+ $node_subscriber->psql('postgres', qq(VACUUM (verbose) public.tab;));
+
+ok($stderr =~ qr/1 are dead but not yet removable/,
+ 'the deleted column is non-removable');
+
+my $log_location = -s $node_subscriber->logfile;
+
+# Wakeup and detach the injection point on the publisher node. The prepared
+# transaction should now commit.
+$node_publisher->safe_psql(
+ 'postgres',
+ "SELECT injection_points_wakeup('commit-after-delay-checkpoint');
+ SELECT injection_points_detach('commit-after-delay-checkpoint');"
+);
+
+# Close the background session on the publisher node
+ok($pub_session->quit, "close publisher session");
+
+# Confirm that the transaction committed
+$result =
+ $node_publisher->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
+is($result, qq(1|2), 'publisher sees the new row');
+
+# Ensure the UPDATE is replayed on subscriber
+$node_publisher->wait_for_catchup($subname);
+
+my $logfile = slurp_file($node_subscriber->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 row \(1, 2\); replica identity \(a\)=\(1\)/,
+ 'update target row was deleted in tab');
+
+# Remember the next transaction ID to be assigned
+my $next_xid =
+ $node_subscriber->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+# Confirm that the xmin value is advanced to the latest nextXid even if there is
+# one transaction on the publisher that has not committed.
+ok( $node_subscriber->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 subscriber"
+);
+
+# Confirm that the dead tuple can be removed now
+($cmdret, $stdout, $stderr) =
+ $node_subscriber->psql('postgres', qq(VACUUM (verbose) public.tab;));
+
+ok($stderr =~ qr/1 removed, 0 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+# Get the commit timestamp for the publisher's update
+my $pub_ts = $node_publisher->safe_psql('postgres',
+ "SELECT pg_xact_commit_timestamp(xmin) from tab where a=1;");
+
+# Check that the commit timestamp for the update on the publisher is later than
+# or equal to the timestamp of the local deletion, as the commit timestamp
+# should be assigned after marking the DELAY_CHKPT_IN_COMMIT flag.
+$result = $node_publisher->safe_psql('postgres',
+ "SELECT '$pub_ts'::timestamp >= '$sub_ts'::timestamp");
+is($result, qq(t),
+ "pub UPDATE's timestamp is later than that of sub's DELETE");
+
+done_testing();
--
2.51.0.windows.1
v2-0001-Fix-conflict-relevant-data-retention-for-prepared.patchapplication/octet-stream; name=v2-0001-Fix-conflict-relevant-data-retention-for-prepared.patchDownload
From 7c4e0f8ee81295ee3868295f06f73957ec874a2c Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Thu, 4 Sep 2025 12:50:59 +0800
Subject: [PATCH v2] Fix conflict-relevant data retention for prepared
transactions
Previously, conflict-relevant data could be prematurely removed before applying
prepared transactions on the publisher that are in the commit critical section.
This occurred because GetOldestActiveTransactionId() was called on the
publisher, which failed to account for the backend executing COMMIT PREPARED, as
this backend does not have an xid stored in PGPROC.
This commit fixes the issue by introducing a new function to traverse global
transactions, identifying prepared transactions in the commit critical section.
---
src/backend/access/transam/twophase.c | 55 +++++++++++++++++++++++++++
src/backend/replication/walsender.c | 12 ++++++
src/include/access/twophase.h | 2 +
3 files changed, 69 insertions(+)
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 7918176fc58..3e20f448787 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -2809,3 +2809,58 @@ LookupGXactBySubid(Oid subid)
return found;
}
+
+/*
+ * TwoPhaseGetXidByLockingProc
+ * Return the oldest transaction ID from prepared transactions that are
+ * currently in the commit critical section.
+ *
+ * This function only considers transactions in the currently connected
+ * database. If no matching transactions are found, it returns
+ * InvalidTransactionId.
+ */
+TransactionId
+TwoPhaseGetOldestXidInCommit(void)
+{
+ TransactionId oldestRunningXid = InvalidTransactionId;
+
+ LWLockAcquire(TwoPhaseStateLock, LW_SHARED);
+
+ for (int i = 0; i < TwoPhaseState->numPrepXacts; i++)
+ {
+ GlobalTransaction gxact = TwoPhaseState->prepXacts[i];
+ PGPROC *commitproc;
+ TransactionId xid;
+
+ if (!gxact->valid)
+ continue;
+
+ if (gxact->locking_backend == INVALID_PROC_NUMBER)
+ continue;
+
+ /*
+ * Get the backend that is handling the transaction. It's safe to
+ * access this backend while holding TwoPhaseStateLock, as the backend
+ * can only be destroyed after either removing or unlocking the
+ * current global transaction, both of which require an exclusive
+ * TwoPhaseStateLock.
+ */
+ commitproc = GetPGProcByNumber(gxact->locking_backend);
+
+ if (MyDatabaseId != commitproc->databaseId)
+ continue;
+
+ if ((commitproc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0)
+ continue;
+
+ xid = XidFromFullTransactionId(gxact->fxid);
+
+ if (!TransactionIdIsValid(oldestRunningXid) ||
+ TransactionIdPrecedes(xid, oldestRunningXid))
+ oldestRunningXid = xid;
+ }
+
+ LWLockRelease(TwoPhaseStateLock);
+
+ return oldestRunningXid;
+}
diff --git a/src/backend/replication/walsender.c b/src/backend/replication/walsender.c
index e3dce9dc68d..59822f22b8d 100644
--- a/src/backend/replication/walsender.c
+++ b/src/backend/replication/walsender.c
@@ -51,6 +51,7 @@
#include "access/timeline.h"
#include "access/transam.h"
+#include "access/twophase.h"
#include "access/xact.h"
#include "access/xlog_internal.h"
#include "access/xlogreader.h"
@@ -2719,6 +2720,7 @@ ProcessStandbyPSRequestMessage(void)
{
XLogRecPtr lsn = InvalidXLogRecPtr;
TransactionId oldestXidInCommit;
+ TransactionId oldestGXidInCommit;
FullTransactionId nextFullXid;
FullTransactionId fullOldestXidInCommit;
WalSnd *walsnd = MyWalSnd;
@@ -2746,6 +2748,16 @@ ProcessStandbyPSRequestMessage(void)
* ones replicated.
*/
oldestXidInCommit = GetOldestActiveTransactionId(true, false);
+ oldestGXidInCommit = TwoPhaseGetOldestXidInCommit();
+
+ /*
+ * Update the oldest xid for standby transmission if an older prepared
+ * transaction exists and is currently in commit phase.
+ */
+ if (TransactionIdIsValid(oldestGXidInCommit) &&
+ TransactionIdPrecedes(oldestGXidInCommit, oldestXidInCommit))
+ oldestXidInCommit = oldestGXidInCommit;
+
nextFullXid = ReadNextFullTransactionId();
fullOldestXidInCommit = FullTransactionIdFromAllowableAt(nextFullXid,
oldestXidInCommit);
diff --git a/src/include/access/twophase.h b/src/include/access/twophase.h
index 509bdad9a5d..64463e9f4af 100644
--- a/src/include/access/twophase.h
+++ b/src/include/access/twophase.h
@@ -68,4 +68,6 @@ extern void TwoPhaseTransactionGid(Oid subid, TransactionId xid, char *gid_res,
int szgid);
extern bool LookupGXactBySubid(Oid subid);
+extern TransactionId TwoPhaseGetOldestXidInCommit(void);
+
#endif /* TWOPHASE_H */
--
2.51.0.windows.1
Coverity is not happy with commit a850be2fe:
/srv/coverity/git/pgsql-git/postgresql/src/backend/replication/logical/worker.c: 3276 in FindDeletedTupleInLocalRel()
3270 * maybe_advance_nonremovable_xid() for details).
3271 */
3272 LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
3273 leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
3274 InvalidOid, false);
3275
CID 1665367: Null pointer dereferences (NULL_RETURNS)
Dereferencing a pointer that might be "NULL" "&leader->relmutex" when calling "tas".
3276 SpinLockAcquire(&leader->relmutex);
3277 oldestxmin = leader->oldest_nonremovable_xid;
3278 SpinLockRelease(&leader->relmutex);
3279 LWLockRelease(LogicalRepWorkerLock);
3280 }
I think Coverity has a point. AFAICS every other call of
logicalrep_worker_find() guards against a NULL result,
so why is it okay for this one to dump core on NULL?
regards, tom lane
On Fri, Sep 5, 2025 at 5:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here are v2 patches which addressed above comments.
I have pushed the first patch. I find that the test can't reliably
fail without a fix. Can you please investigate it?
--
With Regards,
Amit Kapila.
On Mon, Sep 8, 2025 at 3:06 AM Tom Lane <tgl@sss.pgh.pa.us> wrote:
Coverity is not happy with commit a850be2fe:
/srv/coverity/git/pgsql-git/postgresql/src/backend/replication/logical/worker.c: 3276 in FindDeletedTupleInLocalRel()
3270 * maybe_advance_nonremovable_xid() for details).
3271 */
3272 LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
3273 leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
3274 InvalidOid, false);
3275CID 1665367: Null pointer dereferences (NULL_RETURNS)
Dereferencing a pointer that might be "NULL" "&leader->relmutex" when calling "tas".3276 SpinLockAcquire(&leader->relmutex);
3277 oldestxmin = leader->oldest_nonremovable_xid;
3278 SpinLockRelease(&leader->relmutex);
3279 LWLockRelease(LogicalRepWorkerLock);
3280 }I think Coverity has a point. AFAICS every other call of
logicalrep_worker_find() guards against a NULL result,
so why is it okay for this one to dump core on NULL?
Thanks for pointing it out. It was a miss.
I attempted to reproduce a SIGSEGV in this flow. It appears that a
SIGSEGV can occur when the tablesync worker is catching up and is in
FindDeletedTupleInLocalRel() and meanwhile drop-subscription is done
in another session. Here’s the sequence that triggers the issue:
1) Pause the tablesync worker while it's in FindDeletedTupleInLocalRel().
2) Issue a 'DROP SUBSCRIPTION sub'.
3) Allow DropSubscription to proceed to logicalrep_worker_stop() for
the apply worker, but block it using the debugger before it attempts
to stop the tablesync worker.
4) Simultaneously, hold the launcher process using the debugger before
it can restart the apply worker.
5) Now, resume the tablesync worker. It ends up with a NULL leader
worker and hits a SIGSEGV.
Since this issue can be reliably reproduced with a simple DROP
SUBSCRIPTION, I thought it would be appropriate to add the new error
as a user-facing error.
Additionally, the issue can also be reproduced if the apply worker is
forcefully made to error out in wait_for_relation_state_change() while
the tablesync worker is applying changes and is in
FindDeletedTupleInLocalRel().
Attached a patch to address the issue.
thanks
Shveta
Attachments:
v1-0001-Add-NULL-check-for-apply-worker-in-update-deleted.patchapplication/octet-stream; name=v1-0001-Add-NULL-check-for-apply-worker-in-update-deleted.patchDownload
From 63a769cb1c9eb06a953c3c2e8ff2e570c26c389f Mon Sep 17 00:00:00 2001
From: Shveta Malik <shveta.malik@gmail.com>
Date: Mon, 8 Sep 2025 15:49:02 +0530
Subject: [PATCH v1] Add NULL check for apply worker in update-deleted
Add NULL check for leader apply worker in the update-deleted detection
flow to prevent SIGSEGV when the tablesync worker is applying changes and
attempting to locate a deleted row.
---
src/backend/replication/logical/worker.c | 6 ++++++
1 file changed, 6 insertions(+)
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c0f6bef5c28..e30b278109a 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -3272,6 +3272,12 @@ FindDeletedTupleInLocalRel(Relation localrel, Oid localidxoid,
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
leader = logicalrep_worker_find(MyLogicalRepWorker->subid,
InvalidOid, false);
+ if (!leader)
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not detect conflict as the leader apply worker has exited")));
+ }
SpinLockAcquire(&leader->relmutex);
oldestxmin = leader->oldest_nonremovable_xid;
--
2.34.1
On Monday, September 8, 2025 3:13 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 5, 2025 at 5:03 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here are v2 patches which addressed above comments.
I have pushed the first patch. I find that the test can't reliably fail without a fix.
Can you please investigate it?
Thank you for catching this issue. I confirmed that the test may have tested
VACCUM before slot.xmin was advanced. Therefore, to improve the test, I modified
test to wait for the publisher's request message appearing twice, as after the
fix, the apply worker should keep waiting for publisher status until the
prepared txn is committed.
Also, to reduce test time, I moved the test into the existing 035 test.
Here is the updated test.
Best Regards,
Hou zj
Attachments:
v3-0001-Add-a-race-condition-test.patchapplication/octet-stream; name=v3-0001-Add-a-race-condition-test.patchDownload
From fb1cca76163226e450d5e04b30e4e867446d72d0 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 8 Sep 2025 18:25:00 +0800
Subject: [PATCH v3] Add a race condition test
An injection point is introduced along with a tap-test to verify that the
conflict-relevant is not prematurely removed when a concurrent prepared
transaction is being committed on the publisher. Also, verify that the
DELAY_CHKPT_IN_COMMIT marking and timestamp acquisition occur correctly.
---
src/backend/access/transam/twophase.c | 6 +
src/test/subscription/Makefile | 4 +-
src/test/subscription/meson.build | 5 +-
src/test/subscription/t/035_conflicts.pl | 160 +++++++++++++++++++++++
4 files changed, 173 insertions(+), 2 deletions(-)
diff --git a/src/backend/access/transam/twophase.c b/src/backend/access/transam/twophase.c
index 3e20f448787..d8e2fce2c99 100644
--- a/src/backend/access/transam/twophase.c
+++ b/src/backend/access/transam/twophase.c
@@ -103,6 +103,7 @@
#include "storage/proc.h"
#include "storage/procarray.h"
#include "utils/builtins.h"
+#include "utils/injection_point.h"
#include "utils/memutils.h"
#include "utils/timestamp.h"
@@ -2332,12 +2333,17 @@ RecordTransactionCommitPrepared(TransactionId xid,
replorigin = (replorigin_session_origin != InvalidRepOriginId &&
replorigin_session_origin != DoNotReplicateId);
+ /* Load the injection point before entering the critical section */
+ INJECTION_POINT_LOAD("commit-after-delay-checkpoint");
+
START_CRIT_SECTION();
/* See notes in RecordTransactionCommit */
Assert((MyProc->delayChkptFlags & DELAY_CHKPT_IN_COMMIT) == 0);
MyProc->delayChkptFlags |= DELAY_CHKPT_IN_COMMIT;
+ INJECTION_POINT_CACHED("commit-after-delay-checkpoint", NULL);
+
/*
* Ensures the DELAY_CHKPT_IN_COMMIT flag write is globally visible before
* commit time is written.
diff --git a/src/test/subscription/Makefile b/src/test/subscription/Makefile
index 50b65d8f6ea..9d97e7d5c0d 100644
--- a/src/test/subscription/Makefile
+++ b/src/test/subscription/Makefile
@@ -13,9 +13,11 @@ subdir = src/test/subscription
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-EXTRA_INSTALL = contrib/hstore
+EXTRA_INSTALL = contrib/hstore \
+ src/test/modules/injection_points
export with_icu
+export enable_injection_points
check:
$(prove_check)
diff --git a/src/test/subscription/meson.build b/src/test/subscription/meson.build
index 586ffba434e..20b4e523d93 100644
--- a/src/test/subscription/meson.build
+++ b/src/test/subscription/meson.build
@@ -5,7 +5,10 @@ tests += {
'sd': meson.current_source_dir(),
'bd': meson.current_build_dir(),
'tap': {
- 'env': {'with_icu': icu.found() ? 'yes' : 'no'},
+ 'env': {
+ 'with_icu': icu.found() ? 'yes' : 'no',
+ 'enable_injection_points': get_option('injection_points') ? 'yes' : 'no',
+ },
'tests': [
't/001_rep_changes.pl',
't/002_types.pl',
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index e06429c288f..2dc93defc34 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -415,6 +415,166 @@ $node_B->safe_psql('postgres', "ALTER PUBLICATION tap_pub_B ADD TABLE tab");
$node_A->safe_psql('postgres',
"ALTER SUBSCRIPTION $subname_AB REFRESH PUBLICATION WITH (copy_data = false)");
+###############################################################################
+# Test that publisher's transactions marked with DELAY_CHKPT_IN_COMMIT prevent
+# concurrently deleted tuples on the subscriber from being removed. This test
+# also acts as a safeguard to prevent developers from moving the commit
+# timestamp acquisition before marking DELAY_CHKPT_IN_COMMIT in
+# RecordTransactionCommitPrepared.
+###############################################################################
+
+my $injection_points_supported = $node_B->check_extension('injection_points');
+
+# This test depends on an injection point to block the prepared transaction
+# commit after marking DELAY_CHKPT_IN_COMMIT flag.
+if ($injection_points_supported != 0)
+{
+ $node_B->append_conf('postgresql.conf',
+ "shared_preload_libraries = 'injection_points'
+ max_prepared_transactions = 1");
+ $node_B->restart;
+
+ # Disable the subscription on Node B for testing only one-way
+ # replication.
+ $node_B->psql('postgres', "ALTER SUBSCRIPTION $subname_BA DISABLE;");
+
+ # Wait for the apply worker to stop
+ $node_B->poll_query_until('postgres',
+ "SELECT count(*) = 0 FROM pg_stat_activity WHERE backend_type = 'logical replication apply worker'"
+ );
+
+ # Truncate the table to cleanup existing dead rows in the table. Then insert
+ # a new row.
+ $node_B->safe_psql(
+ 'postgres', qq(
+ TRUNCATE tab;
+ INSERT INTO tab VALUES(1, 1);
+ ));
+
+ $node_B->wait_for_catchup($subname_AB);
+
+ # Create the injection_points extension on the publisher node and attach to the
+ # commit-after-delay-checkpoint injection point.
+ $node_B->safe_psql(
+ 'postgres',
+ "CREATE EXTENSION injection_points;
+ SELECT injection_points_attach('commit-after-delay-checkpoint', 'wait');"
+ );
+
+ # Start a background session on the publisher node to perform an update and
+ # pause at the injection point.
+ my $pub_session = $node_B->background_psql('postgres');
+ $pub_session->query_until(
+ qr/starting_bg_psql/,
+ q{
+ \echo starting_bg_psql
+ BEGIN;
+ UPDATE tab SET b = 2 WHERE a = 1;
+ PREPARE TRANSACTION 'txn_with_later_commit_ts';
+ COMMIT PREPARED 'txn_with_later_commit_ts';
+ }
+ );
+
+ # Confirm the update is suspended
+ $result =
+ $node_B->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
+ is($result, qq(1|1), 'publisher sees the old row');
+
+ # Delete the row on the subscriber. The deleted row should be retained due to a
+ # transaction on the publisher, which is currently marked with the
+ # DELAY_CHKPT_IN_COMMIT flag.
+ $node_A->safe_psql('postgres', "DELETE FROM tab WHERE a = 1;");
+
+ # Get the commit timestamp for the delete
+ my $sub_ts = $node_A->safe_psql('postgres',
+ "SELECT timestamp FROM pg_last_committed_xact();");
+
+ $log_location = -s $node_A->logfile;
+
+ # Confirm that the apply worker keeps requesting publisher status, while
+ # awaiting the prepared transaction to commit. Thus, the request log should
+ # appear more than once.
+ $node_A->wait_for_log(
+ qr/sending publisher status request message/,
+ $log_location);
+
+ $log_location = -s $node_A->logfile;
+
+ $node_A->wait_for_log(
+ qr/sending publisher status request message/,
+ $log_location);
+
+ # Confirm that the dead tuple cannot be removed
+ ($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');
+
+ $log_location = -s $node_A->logfile;
+
+ # Wakeup and detach the injection point on the publisher node. The prepared
+ # transaction should now commit.
+ $node_B->safe_psql(
+ 'postgres',
+ "SELECT injection_points_wakeup('commit-after-delay-checkpoint');
+ SELECT injection_points_detach('commit-after-delay-checkpoint');"
+ );
+
+ # Close the background session on the publisher node
+ ok($pub_session->quit, "close publisher session");
+
+ # Confirm that the transaction committed
+ $result =
+ $node_B->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
+ is($result, qq(1|2), 'publisher sees the new row');
+
+ # Ensure the UPDATE is replayed on subscriber
+ $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 row \(1, 2\); replica identity full \(1, 1\)/,
+ 'update target row was deleted in tab');
+
+ # Remember the next transaction ID to be assigned
+ $next_xid =
+ $node_A->safe_psql('postgres', "SELECT txid_current() + 1;");
+
+ # Confirm that the xmin value is advanced to the latest nextXid even if there is
+ # one transaction on the publisher that has not committed.
+ 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 subscriber"
+ );
+
+ # Confirm that the dead tuple can be removed now
+ ($cmdret, $stdout, $stderr) =
+ $node_A->psql('postgres', qq(VACUUM (verbose) public.tab;));
+
+ ok($stderr =~ qr/1 removed, 0 remain, 0 are dead but not yet removable/,
+ 'the deleted column is removed');
+
+ # Get the commit timestamp for the publisher's update
+ my $pub_ts = $node_B->safe_psql('postgres',
+ "SELECT pg_xact_commit_timestamp(xmin) from tab where a=1;");
+
+ # Check that the commit timestamp for the update on the publisher is later than
+ # or equal to the timestamp of the local deletion, as the commit timestamp
+ # should be assigned after marking the DELAY_CHKPT_IN_COMMIT flag.
+ $result = $node_B->safe_psql('postgres',
+ "SELECT '$pub_ts'::timestamp >= '$sub_ts'::timestamp");
+ is($result, qq(t),
+ "pub UPDATE's timestamp is later than that of sub's DELETE");
+
+ # Re-enable the subscription for further tests
+ $node_B->psql('postgres', "ALTER SUBSCRIPTION $subname_BA ENABLE;");
+}
+
###############################################################################
# Check that dead tuple retention stops due to the wait time surpassing
# max_retention_duration.
--
2.51.0.windows.1
On Tuesday, September 2, 2025 6:00 PM shveta malik <shveta.malik@gmail.com> wrote:
On Mon, Sep 1, 2025 at 5:45 PM shveta malik <shveta.malik@gmail.com>
wrote:Here is V70 patch set.
The patch v70-0001 looks good to me. Verified, all the old issues are resolved.
Will resume review of v70-0002 now.
Please find a few comments on v70-0002:
1) - * Note: Retention won't be resumed automatically. The user must manually - * disable retain_dead_tuples and re-enable it after confirming that the - * replication slot maintained by the launcher has been dropped. + * The retention will resume automatically if the worker has confirmed + that the + * retention duration is now within the max_retention_duration.Do we need this comment atop stop as it does not directly concern stop? Isn't
the details regarding RDT_RESUME_CONFLICT_INFO_RETENTION
in the file-header section sufficient?
Agreed. I removed this comment.
2) + /* Stop retention if not yet */ + if (MySubscription->retentionactive) + { + rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;- /* process the next phase */ - process_rdt_phase_transition(rdt_data, false); + /* process the next phase */ + process_rdt_phase_transition(rdt_data, false); } + + reset_retention_data_fields(rdt_data);should_stop_conflict_info_retention() does reset_retention_data_fields
everytime when wait-time exceeds the limit, and when it actually stops i.e.
calls stop_conflict_info_retention through phase change; the stop function
also does reset_retention_data_fields and calls process_rdt_phase_transition.
Can we optimize this code part by consolidating the
reset_retention_data_fields() and
process_rdt_phase_transition() calls into
should_stop_conflict_info_retention() itself, eliminating redundancy?
Agreed. I improved the code here.
3)
Shall we update 035_conflicts.pl to have a resume test by setting
max_retention_duration to 0 after stop-retention test?
Added.
4) + subscription. The retention will be automatically resumed once at least + one apply worker confirms that the retention duration is within the + specified limit, or a new subscription is created with + <literal>retain_dead_tuples = true</literal>, or the user manually + re-enables <literal>retain_dead_tuples</literal>.Shall we rephrase it slightly to:
Retention will automatically resume when at least one apply worker confirms
that the retention duration is within the specified limit, or when a new
subscription is created with <literal>retain_dead_tuples = true</literal>.
Alternatively, retention can be manually resumed by re-enabling
<literal>retain_dead_tuples</literal>.
Changed as suggested.
Here is V71 patch set which addressed above comments and [1]/messages/by-id/CAJpy0uC8w442wGEJ0gyR23ojAyvd-s_g-m8fUbixy0V9yOmrcg@mail.gmail.com.
[1]: /messages/by-id/CAJpy0uC8w442wGEJ0gyR23ojAyvd-s_g-m8fUbixy0V9yOmrcg@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v71-0002-Add-a-dead_tuple_retention_active-column-in-pg_s.patchapplication/octet-stream; name=v71-0002-Add-a-dead_tuple_retention_active-column-in-pg_s.patchDownload
From e7f4a93fdeea15489fbed4b3404ffd08a958af69 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Mon, 25 Aug 2025 10:03:28 +0800
Subject: [PATCH v71 2/2] Add a dead_tuple_retention_active column in
pg_stat_subscription
To monitor worker's conflict retention status, this patch also introduces a new
column 'dead_tuple_retention_active' 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_dead_tuples is enabled
for the associated subscription, and the retention duration for conflict
detection by the apply worker has not exceeded max_retention_duration.
---
doc/src/sgml/monitoring.sgml | 12 ++++++++++++
src/backend/catalog/system_views.sql | 3 ++-
src/backend/replication/logical/launcher.c | 18 +++++++++++++++++-
src/include/catalog/pg_proc.dat | 6 +++---
src/test/regress/expected/rules.out | 5 +++--
src/test/subscription/t/035_conflicts.pl | 12 ++++++++++++
6 files changed, 49 insertions(+), 7 deletions(-)
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index 3f4a27a736e..96270f03bf2 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -2114,6 +2114,18 @@ description | Waiting for a newly initialized WAL file to reach durable storage
sender; NULL for parallel apply workers
</para></entry>
</row>
+
+ <row>
+ <entry role="catalog_table_entry"><para role="column_definition">
+ <structfield>dead_tuple_retention_active</structfield> <type>boolean</type>
+ </para>
+ <para>
+ True if <link linkend="sql-createsubscription-params-with-retain-dead-tuples"><literal>retain_dead_tuples</literal></link>
+ is enabled and the retention duration for information used in conflict detection is
+ within <link linkend="sql-createsubscription-params-with-max-retention-duration"><literal>max_retention_duration</literal></link>; NULL for
+ parallel apply workers and table synchronization workers.
+ </para></entry>
+ </row>
</tbody>
</tgroup>
</table>
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index c77fa0234bb..b47109ab11d 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -996,7 +996,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.dead_tuple_retention_active
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 226421e3dfa..cd54ac7cbf2 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1593,7 +1593,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;
@@ -1670,6 +1670,22 @@ pg_stat_get_subscription(PG_FUNCTION_ARGS)
elog(ERROR, "unknown worker type");
}
+ /*
+ * Use the worker's oldest_nonremovable_xid instead of
+ * pg_subscription.subretentionactive to determine whether retention
+ * is active, as retention resumption might not be complete even when
+ * subretentionactive is set to true; this is because the launcher
+ * assigns the initial oldest_nonremovable_xid after the apply worker
+ * updates the catalog (see resume_conflict_info_retention).
+ *
+ * Only the leader apply worker manages conflict retention (see
+ * maybe_advance_nonremovable_xid() for details).
+ */
+ if (!isParallelApplyWorker(&worker) && !isTablesyncWorker(&worker))
+ values[10] = TransactionIdIsValid(worker.oldest_nonremovable_xid);
+ else
+ nulls[10] = true;
+
tuplestore_putvalues(rsinfo->setResult, rsinfo->setDesc,
values, nulls);
diff --git a/src/include/catalog/pg_proc.dat b/src/include/catalog/pg_proc.dat
index 118d6da1ace..3810f3883b7 100644
--- a/src/include/catalog/pg_proc.dat
+++ b/src/include/catalog/pg_proc.dat
@@ -5696,9 +5696,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,dead_tuple_retention_active}',
prosrc => 'pg_stat_get_subscription' },
{ oid => '2026', descr => 'statistics: current backend PID',
proname => 'pg_backend_pid', provolatile => 's', proparallel => 'r',
diff --git a/src/test/regress/expected/rules.out b/src/test/regress/expected/rules.out
index 35e8aad7701..183fc193ad3 100644
--- a/src/test/regress/expected/rules.out
+++ b/src/test/regress/expected/rules.out
@@ -2174,9 +2174,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.dead_tuple_retention_active
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, dead_tuple_retention_active) ON ((st.subid = su.oid)));
pg_stat_subscription_stats| SELECT ss.subid,
s.subname,
ss.apply_error_count,
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 947ea131c4d..f53209120a3 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -580,6 +580,10 @@ if ($injection_points_supported != 0)
# max_retention_duration.
###############################################################################
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains dead tuples');
+
# Create a physical slot
$node_B->safe_psql('postgres',
"SELECT * FROM pg_create_physical_replication_slot('blocker');");
@@ -622,6 +626,10 @@ $result = $node_A->safe_psql('postgres',
"SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
is($result, qq(f), 'retention is inactive');
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(f), 'worker on node A does not retain dead tuples');
+
# Drop the physical slot and reset the synchronized_standby_slots setting
$node_B->safe_psql('postgres',
"SELECT * FROM pg_drop_replication_slot('blocker');");
@@ -655,6 +663,10 @@ $result = $node_A->safe_psql('postgres',
"SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
is($result, qq(t), 'retention is active');
+$result = $node_A->safe_psql('postgres',
+ "SELECT dead_tuple_retention_active FROM pg_stat_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'worker on node A retains dead tuples');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
v71-0001-Allow-conflict-relevant-data-retention-to-resume.patchapplication/octet-stream; name=v71-0001-Allow-conflict-relevant-data-retention-to-resume.patchDownload
From f77d62a647e369916baf811d0c8e2c986ec4cb12 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 9 Sep 2025 11:03:36 +0800
Subject: [PATCH v71 1/2] Allow conflict-relevant data retention to resume
This commit enables automatic recovery of conflict-relevant data retention for a
subscription. If the retention duration for a subscription previously exceeded
the max_retention_duration and caused retention to stop, the retention can
resume once the duration falls within the acceptable limits.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/replication/logical/launcher.c | 48 ++++-
src/backend/replication/logical/worker.c | 202 ++++++++++++++++++---
src/include/replication/worker_internal.h | 6 +
src/test/subscription/t/035_conflicts.pl | 27 +++
5 files changed, 256 insertions(+), 36 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ed82cf1809e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will automatically resume when at least one
+ apply worker confirms that the retention duration is within the
+ specified limit, or when a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>. Alternatively, retention
+ can be manually resumed by re-enabling <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add2e2e066c..226421e3dfa 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,7 +101,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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
static void init_conflict_slot_xmin(void);
@@ -468,6 +470,7 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->wait_for_initial_xid = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1270,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1382,11 +1383,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * Additionally, if an apply worker has an invalid XID and is requesting to
+ * resume retention, assign the slot's xmin value to it.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool wait_for_xid;
Assert(worker != NULL);
@@ -1398,16 +1404,42 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ wait_for_xid = worker->wait_for_initial_xid;
SpinLockRelease(&worker->relmutex);
/*
- * Return if the apply worker has stopped retention concurrently.
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if
+ * requested. This ensures the apply worker continues to maintain the
+ * oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (wait_for_xid)
+ {
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ Assert(TransactionIdIsValid(nonremovable_xid));
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return if the apply worker has stopped retention concurrently and has not
+ * yet resumed.
*
* Although this function is invoked only when retentionactive is true,
* the apply worker might stop retention after the launcher fetches the
* retentionactive flag.
*/
- if (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ if (!can_update_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index c0f6bef5c28..2401ece4558 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION
+ * at any phase if the retention has been stopped, but max_retention_duration
+ * is now set to 0.
+ *
* 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) ->
@@ -382,7 +395,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -569,6 +583,10 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4340,6 +4358,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4362,10 +4387,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4394,6 +4415,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4667,6 +4691,18 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4696,10 +4732,6 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* If retention should be stopped, transition to the
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
- *
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4730,7 +4762,14 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase and
+ * recalculate the wait time.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ else
+ reset_retention_data_fields(rdt_data);
/* process the next phase */
process_rdt_phase_transition(rdt_data, false);
@@ -4743,6 +4782,131 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ update_retention_status(false);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !MySubscription->retentionactive;
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ update_retention_status(true);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = false;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Skip the update if currently within an existing transaction.
+ */
+static void
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4760,26 +4924,16 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
}
/*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index de003802612..4a90f130e79 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -100,6 +100,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker is resuming retention and is waiting
+ * for the launcher to initialize oldest_nonremovable_xid.
+ */
+ bool wait_for_initial_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index db0d5b464e8..947ea131c4d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -628,6 +628,33 @@ $node_B->safe_psql('postgres',
$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
$node_B->reload;
+###############################################################################
+# Check that dead tuple retention resumes when the max_retention_duration is set
+# 0.
+###############################################################################
+
+$log_offset = -s $node_A->logfile;
+
+# Set max_retention_duration to 0
+$node_A->safe_psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 0);");
+
+# Confirm that the retention resumes
+$node_A->wait_for_log(
+ qr/logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
+.*DETAIL:.* Retention of information used for conflict detection is now indefinite.*/,
+ $log_offset);
+
+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");
+
+$result = $node_A->safe_psql('postgres',
+ "SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'retention is active');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
On Wednesday, September 3, 2025 12:19 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Sep 2, 2025 at 3:30 PM shveta malik <shveta.malik@gmail.com>
wrote:Here is V70 patch set.
Please find a few comments on v70-003:
1)
Doc of dead_tuple_retention_active says:
True if retain_dead_tuples is enabled and the retention duration for information
used in conflict detection is within max_retention_durationDoc of subretentionactive says:
The retention status of information (e.g., dead tuples, commit timestamps, and
origins) useful for conflict detection. True if retain_dead_tuples is enabled, and
the retention duration has not exceeded max_retention_duration, when
defined.There is hardly any difference between the two. Do we really need to have
'dead_tuple_retention_active' when we already have 'subretentionactive'?
I felt the retentionactive in pg_subscription can be considered as a internal
flag and users might prefer to access public views over catalogs when possible,
so I'm keeping this new column in the view for now. However, if we find it's not
worthwhile, we can discuss removing this patch.
2)
Doc wise, there is no difference between the two, but there is a small window
when sub's subretentionactive will show true while stat's
dead_tuple_retention_active will show false. This will be when worker is
waiting for the launcher to assign its oldest-xid after it has marked itself as
'resuming'.
If we decide to retain 'dead_tuple_retention_active', then do we need to
indicate the small difference between the 2 fields in the doc?
Yes, this is also the difference.
I tried to explain the difference in a user visible way, that is, if
dead_tuple_retention_active is true, then update_deleted detection is enabled
(pg_subscription.retentionactive does not ensure this).
3)
We can add a test when we stop-retention to see if this is showing false.
Currently there are 2 places in the test where we check this field to see if it is
true. I think we can shift both in the same test. One check before stop-retention,
one check after stop-retention.
Changed.
Best Regards,
Hou zj
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is V71 patch set which addressed above comments and [1].
Thank You for the patches. Please find a few comments on 001:
1)
In compute_min_nonremovable_xid, when 'wait_for_xid' is true, we
should have Assert(!worker->oldest_nonremovable_xid) to ensure it is
always Invalid if reached here.
Or we can rewrite the current if-else-if code logic based on worker's
oldest-xid as main criteria as that will be NULL in both the blocks:
if (!TransactionIdIsValid(nonremovable_xid))
{
/* resume case */
if(wait_for_xid)
set worker's oldest-xid using slot's xmin
else
/* stop case */
return;
}
It will be slightly easier to understand.
2)
In stop_conflict_info_retention(), there may be a case where due to an
ongoing transaction, it could not update retentionactive to false. But
even in such cases, the function still sets oldest_nonremovable_xid to
Invalid, which seems wrong.
3)
Similar in resume flow, it still sets wait_for_initial_xid=true even
when it could not update retentionactive=true.
4)
resume_conflict_info_retention():
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
I think we should have
'Assert(MyLogicalRepWorker->wait_for_initial_xid)' before 'return'
here.
thanks
Shveta
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is V71 patch set which addressed above comments and [1].
IIUC, this patch after stopping the retention, it immediately starts
retrying to resume by transitioning through various phases. This can
consume CPU and network resources, if the apply_worker takes a long
time to catch up.
Few other comments:
1.
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
I understand the reason why we get assigned the worker's
non_removable_xid from launcher but all this makes the code complex to
understand. Isn't there any other way to achieve it? One naïve and
inefficient way could be to just restart the worker probably after
updating its retentionactive flag. I am not suggesting to make this
change but just a brainstorming point.
2. The function should_stop_conflict_info_retention() is invoked from
wait_for_local_flush() and then it can lead further state
transitioning which doesn't appear neat and makes code difficult to
understand.
3.
+ /*
+ * If conflict info retention was previously stopped due to a timeout, and
+ * the time required to advance the non-removable transaction ID has now
+ * decreased to within acceptable limits, resume the rentention.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ return;
+ }
In this check, where do we check the time has come in acceptable
range? Can you update comments to make it clear?
--
With Regards,
Amit Kapila.
On Tuesday, September 9, 2025 7:01 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is V71 patch set which addressed above comments and [1].
IIUC, this patch after stopping the retention, it immediately starts retrying to
resume by transitioning through various phases. This can consume CPU and
network resources, if the apply_worker takes a long time to catch up.
I agree. I think one way is to increase the interval in each cycle when the retention has
been stopped and the worker is retrying to resume the retention. I have
updated the patch for the same.
Few other comments: 1. + /* + * Return if the launcher has not initialized oldest_nonremovable_xid. + * + * It might seem feasible to directly check the conflict detection + * slot.xmin instead of relying on the launcher to assign the worker's + * oldest_nonremovable_xid; however, that could lead to a race + condition + * where slot.xmin is set to InvalidTransactionId immediately after the + * check. In such cases, oldest_nonremovable_xid would no longer be + * protected by a replication slot and could become unreliable if a + * wraparound occurs. + */ + if (!TransactionIdIsValid(nonremovable_xid)) + return;I understand the reason why we get assigned the worker's non_removable_xid
from launcher but all this makes the code complex to understand. Isn't there
any other way to achieve it? One naïve and inefficient way could be to just
restart the worker probably after updating its retentionactive flag. I am not
suggesting to make this change but just a brainstorming point.
I'll keep thinking about it, and for now, I've added a comment mentioning
that rebooting is a simpler solution.
2. The function should_stop_conflict_info_retention() is invoked from
wait_for_local_flush() and then it can lead further state transitioning which
doesn't appear neat and makes code difficult to understand.
I changed the logic to avoid proceeding to next phase when
the retention is stopped.
3. + /* + * If conflict info retention was previously stopped due to a timeout, + and + * the time required to advance the non-removable transaction ID has + now + * decreased to within acceptable limits, resume the rentention. + */ + if (!MySubscription->retentionactive) + { + rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION; + process_rdt_phase_transition(rdt_data, false); return; }In this check, where do we check the time has come in acceptable range? Can
you update comments to make it clear?
I updated the comments to mention that the code can reach here only
when the time is within max_retention_duration.
Here is the V72 patch set which addressed above and Shveta's comments[1]/messages/by-id/CAJpy0uDw7SmCN_jOvpNUzo_sE4ZsgpqQ5_vHLjm4aCm10eBApA@mail.gmail.com.
[1]: /messages/by-id/CAJpy0uDw7SmCN_jOvpNUzo_sE4ZsgpqQ5_vHLjm4aCm10eBApA@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v72-0001-Allow-conflict-relevant-data-retention-to-resume.patchapplication/octet-stream; name=v72-0001-Allow-conflict-relevant-data-retention-to-resume.patchDownload
From 47f0fb869551667ccb2de3e9155a6adba9c11fa1 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 9 Sep 2025 11:03:36 +0800
Subject: [PATCH v72] Allow conflict-relevant data retention to resume
This commit enables automatic recovery of conflict-relevant data retention for a
subscription. If the retention duration for a subscription previously exceeded
the max_retention_duration and caused retention to stop, the retention can
resume once the duration falls within the acceptable limits.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/replication/logical/launcher.c | 49 +++-
src/backend/replication/logical/worker.c | 262 ++++++++++++++++++---
src/include/replication/worker_internal.h | 6 +
src/test/subscription/t/035_conflicts.pl | 27 +++
5 files changed, 302 insertions(+), 51 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ed82cf1809e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will automatically resume when at least one
+ apply worker confirms that the retention duration is within the
+ specified limit, or when a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>. Alternatively, retention
+ can be manually resumed by re-enabling <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add2e2e066c..7bccc2ee796 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -101,7 +101,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 compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin);
+static void compute_min_nonremovable_xid(LogicalRepWorker *worker,
+ bool can_update_xmin,
+ TransactionId *xmin);
static bool acquire_conflict_slot_if_exists(void);
static void update_conflict_slot_xmin(TransactionId new_xmin);
static void init_conflict_slot_xmin(void);
@@ -468,6 +470,7 @@ retry:
worker->oldest_nonremovable_xid = retain_dead_tuples
? MyReplicationSlot->data.xmin
: InvalidTransactionId;
+ worker->wait_for_initial_xid = false;
worker->last_lsn = InvalidXLogRecPtr;
TIMESTAMP_NOBEGIN(worker->last_send_time);
TIMESTAMP_NOBEGIN(worker->last_recv_time);
@@ -1270,10 +1273,8 @@ ApplyLauncherMain(Datum main_arg)
* required for conflict detection among all running apply
* workers.
*/
- if (sub->retaindeadtuples &&
- sub->retentionactive &&
- can_update_xmin)
- compute_min_nonremovable_xid(w, &xmin);
+ if (sub->retaindeadtuples && sub->retentionactive)
+ compute_min_nonremovable_xid(w, can_update_xmin, &xmin);
/* worker is running already */
continue;
@@ -1382,11 +1383,16 @@ ApplyLauncherMain(Datum main_arg)
* Determine the minimum non-removable transaction ID across all apply workers
* for subscriptions that have retain_dead_tuples enabled. Store the result
* in *xmin.
+ *
+ * Additionally, if an apply worker has an invalid XID and is requesting to
+ * resume retention, assign the slot's xmin value to it.
*/
static void
-compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
+compute_min_nonremovable_xid(LogicalRepWorker *worker, bool can_update_xmin,
+ TransactionId *xmin)
{
TransactionId nonremovable_xid;
+ bool wait_for_xid;
Assert(worker != NULL);
@@ -1398,16 +1404,43 @@ compute_min_nonremovable_xid(LogicalRepWorker *worker, TransactionId *xmin)
SpinLockAcquire(&worker->relmutex);
nonremovable_xid = worker->oldest_nonremovable_xid;
+ wait_for_xid = worker->wait_for_initial_xid;
SpinLockRelease(&worker->relmutex);
/*
- * Return if the apply worker has stopped retention concurrently.
+ * Assign slot.xmin to the apply worker's oldest_nonremovable_xid if
+ * requested. This ensures the apply worker continues to maintain the
+ * oldest_nonremovable_xid (see resume_conflict_info_retention).
+ */
+ if (wait_for_xid)
+ {
+ Assert(!TransactionIdIsValid(nonremovable_xid) &&
+ TransactionIdIsValid(MyReplicationSlot->data.xmin));
+
+ nonremovable_xid = MyReplicationSlot->data.xmin;
+
+ SpinLockAcquire(&worker->relmutex);
+ worker->oldest_nonremovable_xid = nonremovable_xid;
+ SpinLockRelease(&worker->relmutex);
+
+ /* Notify the apply worker to start the next cycle of management */
+ LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
+ logicalrep_worker_wakeup_ptr(worker);
+ LWLockRelease(LogicalRepWorkerLock);
+ }
+
+ /*
+ * Return if the apply worker has stopped retention concurrently and has
+ * not yet resumed.
*
* Although this function is invoked only when retentionactive is true,
* the apply worker might stop retention after the launcher fetches the
* retentionactive flag.
*/
- if (!TransactionIdIsValid(nonremovable_xid))
+ else if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ if (!can_update_xmin)
return;
if (!TransactionIdIsValid(*xmin) ||
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ee6ac22329f..c5e9b1bd4a1 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,19 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and we wait for the launcher to initialize the
+ * oldest_nonremovable_xid before proceeding to RDT_GET_CANDIDATE_XID phase.
+ * Note that the state could transition to RDT_RESUME_CONFLICT_INFO_RETENTION
+ * at any phase if the retention has been stopped, but max_retention_duration
+ * is now set to 0.
+ *
* 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) ->
@@ -381,7 +394,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -568,6 +582,10 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
@@ -4345,6 +4363,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4367,10 +4392,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4399,6 +4420,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4672,6 +4696,22 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * Reaching this point implies should_stop_conflict_info_retention()
+ * returned false earlier, indicating that the most recent duration for
+ * advancing the non-removable transaction ID is within the
+ * max_retention_duration.
+ *
+ * Therefore, if conflict info retention was previously halted due to a
+ * timeout, proceed to resume retention now.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ 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
@@ -4701,10 +4741,6 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* If retention should be stopped, transition to the
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
- *
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4735,10 +4771,20 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ /* Stop retention if not yet */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ return true;
+ }
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
+ /*
+ * If retention has been stopped, reset to the initial phase to retry all
+ * phases. This is required to recalculate the current wait time and
+ * resume retention if the time falls within max_retention_duration.
+ */
+ reset_retention_data_fields(rdt_data);
return true;
}
@@ -4748,6 +4794,146 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Return if unable to update subretentionactive (see
+ * update_retention_status).
+ */
+ if (!update_retention_status(false))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !MySubscription->retentionactive;
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ TransactionId nonremovable_xid;
+
+ /* Update the pg_subscription.retentionactive if not yet */
+ if (!MySubscription->retentionactive)
+ {
+ /*
+ * Return if unable to update subretentionactive (see
+ * update_retention_status).
+ */
+ if (!update_retention_status(true))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = true;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+ }
+
+ Assert(MyLogicalRepWorker->wait_for_initial_xid);
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ nonremovable_xid = MyLogicalRepWorker->oldest_nonremovable_xid;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Return if the launcher has not initialized oldest_nonremovable_xid.
+ *
+ * It might seem feasible to directly check the conflict detection
+ * slot.xmin instead of relying on the launcher to assign the worker's
+ * oldest_nonremovable_xid; however, that could lead to a race condition
+ * where slot.xmin is set to InvalidTransactionId immediately after the
+ * check. In such cases, oldest_nonremovable_xid would no longer be
+ * protected by a replication slot and could become unreliable if a
+ * wraparound occurs.
+ *
+ * XXX An alternative could be directly restarting the worker to ensure
+ * the launcher initializes oldest_nonremovable_xid prior to starting.
+ * However, restarting may not be preferable if initialization can be
+ * managed on-the-fly.
+ */
+ if (!TransactionIdIsValid(nonremovable_xid))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->wait_for_initial_xid = false;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ /*
+ * Proceed to the next phase if either the launcher has initialized
+ * slot.xmin and assigned it to oldest_nonremovable_xid, or retention has
+ * not been stopped yet. The latter situation arises when transitioning
+ * from the RDT_STOP_CONFLICT_INFO_RETENTION phase but subretentionactive
+ * has not been updated due to the inability to start a new transaction
+ * (see stop_conflict_info_retention).
+ */
+ Assert(MySubscription->retentionactive);
+
+ reset_retention_data_fields(rdt_data);
+
+ /* process the next phase */
+ process_rdt_phase_transition(rdt_data, false);
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Returns true upon successful update; however, if currently within an active
+ * transaction, skip the update and return false.
+ */
+static bool
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4755,7 +4941,7 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
* rollback of catalog updates if the application fails subsequently.
*/
if (IsTransactionState())
- return;
+ return false;
StartTransactionCommand();
@@ -4765,26 +4951,18 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
- /* Notify launcher to update the conflict slot */
+ /* Notify launcher to update the xmin of the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
+
+ return true;
}
/*
@@ -4809,19 +4987,20 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
- * If there is no activity on the node, we progressively double the interval
- * used to advance non-removable transaction ID. This helps conserve CPU
- * and network resources when there's little benefit to frequent updates.
+ * If there is no activity on the node or retention has been stopped, we
+ * progressively double the interval used to advance non-removable transaction
+ * ID. This helps conserve CPU and network resources when there's little benefit
+ * to frequent updates.
*
* The interval is capped by the lowest of the following:
* - wal_receiver_status_interval (if set),
* - a default maximum of 3 minutes,
- * - max_retention_duration.
+ * - max_retention_duration (if retention is active).
*
- * This ensures the interval never exceeds the retention boundary, even if
- * other limits are higher. Once activity resumes on the node, the interval
- * is reset to lesser of 100ms and max_retention_duration, allowing timely
- * advancement of non-removable transaction ID.
+ * This ensures the interval never exceeds the retention boundary, even if other
+ * limits are higher. Once activity resumes on the node and the retention is
+ * active, the interval is reset to lesser of 100ms and max_retention_duration,
+ * allowing timely advancement of non-removable transaction ID.
*
* 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.
@@ -4829,7 +5008,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
static void
adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
{
- if (!new_xid_found && rdt_data->xid_advance_interval)
+ if (rdt_data->xid_advance_interval &&
+ (!new_xid_found || !MySubscription->retentionactive))
{
int max_interval = wal_receiver_status_interval
? wal_receiver_status_interval * 1000
@@ -4851,9 +5031,13 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
- /* Ensure the wait time remains within the maximum limit */
- rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
- MySubscription->maxretention);
+ /*
+ * Ensure the wait time remains within the maximum limit when retention is
+ * active.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
diff --git a/src/include/replication/worker_internal.h b/src/include/replication/worker_internal.h
index de003802612..d776949a04e 100644
--- a/src/include/replication/worker_internal.h
+++ b/src/include/replication/worker_internal.h
@@ -100,6 +100,12 @@ typedef struct LogicalRepWorker
*/
TransactionId oldest_nonremovable_xid;
+ /*
+ * Indicates whether the apply worker is resuming retention and is waiting
+ * for the launcher to initialize oldest_nonremovable_xid.
+ */
+ bool wait_for_initial_xid;
+
/* Stats. */
XLogRecPtr last_lsn;
TimestampTz last_send_time;
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index db0d5b464e8..947ea131c4d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -628,6 +628,33 @@ $node_B->safe_psql('postgres',
$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
$node_B->reload;
+###############################################################################
+# Check that dead tuple retention resumes when the max_retention_duration is set
+# 0.
+###############################################################################
+
+$log_offset = -s $node_A->logfile;
+
+# Set max_retention_duration to 0
+$node_A->safe_psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 0);");
+
+# Confirm that the retention resumes
+$node_A->wait_for_log(
+ qr/logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
+.*DETAIL:.* Retention of information used for conflict detection is now indefinite.*/,
+ $log_offset);
+
+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");
+
+$result = $node_A->safe_psql('postgres',
+ "SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'retention is active');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
On Tuesday, September 9, 2025 5:17 PM shveta malik <shveta.malik@gmail.com> wrote:
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is V71 patch set which addressed above comments and [1].
Thank You for the patches. Please find a few comments on 001:
1)
In compute_min_nonremovable_xid, when 'wait_for_xid' is true, we should
have Assert(!worker->oldest_nonremovable_xid) to ensure it is always Invalid
if reached here.
Added.
Or we can rewrite the current if-else-if code logic based on worker's oldest-xid
as main criteria as that will be NULL in both the blocks:if (!TransactionIdIsValid(nonremovable_xid))
{
/* resume case */
if(wait_for_xid)
set worker's oldest-xid using slot's xmin
else
/* stop case */
return;
}
I am personally not in favor this style because it adds more conditions, so I
did not change in this version.
2)
In stop_conflict_info_retention(), there may be a case where due to an ongoing
transaction, it could not update retentionactive to false. But even in such cases,
the function still sets oldest_nonremovable_xid to Invalid, which seems wrong.3)
Similar in resume flow, it still sets wait_for_initial_xid=true even when it could
not update retentionactive=true.
Right, I missed to return when the update fails. Fixed in this version.
4) resume_conflict_info_retention(): + /* + * Return if the launcher has not initialized oldest_nonremovable_xid. + * + */ + if (!TransactionIdIsValid(nonremovable_xid)) + return;I think we should have
'Assert(MyLogicalRepWorker->wait_for_initial_xid)' before 'return'
here.
Added.
Best Regards,
Hou zj
On Wed, Sep 10, 2025 at 9:08 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 9, 2025 7:01 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Few other comments: 1. + /* + * Return if the launcher has not initialized oldest_nonremovable_xid. + * + * It might seem feasible to directly check the conflict detection + * slot.xmin instead of relying on the launcher to assign the worker's + * oldest_nonremovable_xid; however, that could lead to a race + condition + * where slot.xmin is set to InvalidTransactionId immediately after the + * check. In such cases, oldest_nonremovable_xid would no longer be + * protected by a replication slot and could become unreliable if a + * wraparound occurs. + */ + if (!TransactionIdIsValid(nonremovable_xid)) + return;I understand the reason why we get assigned the worker's non_removable_xid
from launcher but all this makes the code complex to understand. Isn't there
any other way to achieve it? One naïve and inefficient way could be to just
restart the worker probably after updating its retentionactive flag. I am not
suggesting to make this change but just a brainstorming point.I'll keep thinking about it, and for now, I've added a comment mentioning
that rebooting is a simpler solution.
Say we have a LW LogicalRepRetentionLock. We acquire it in SHARED mode
as soon as we encounter the first subscription with retain_dead_tuples
set during the traversal of the sublist. We release it only after
updating xmin outside the sublist traversal. We then acquire it in
EXCLUSIVE mode to fetch the resume the retention in worker for the
period till we fetch slot's xmin.
This will close the above race condition but acquiring LWLock while
traversing subscription is not advisable as that will make it
uninterruptible. The other possibility is to use some heavy-weight
lock here, say a lock on pg_subscription catalog but that has a
drawback that it can conflict with DDL operations. Yet another way is
to invent a new lock-type for this.
OTOH, the simple strategy to let apply-worker restart to resume
retention will keep the handling simpler. We do something similar at
the start of apply-worker if we find that some subscription parameter
is changed. I think we need more opinions on this matter.
One other comment:
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ process_rdt_phase_transition(rdt_data, false);
+ return;
+ }
It is better that the caller processes the next phase, otherwise, this
looks a bit ad hoc. Similarly, please check other places.
--
With Regards,
Amit Kapila.
On Monday, September 8, 2025 7:21 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Monday, September 8, 2025 3:13 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:On Fri, Sep 5, 2025 at 5:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:Here are v2 patches which addressed above comments.
I have pushed the first patch. I find that the test can't reliably fail without a fix.
Can you please investigate it?Thank you for catching this issue. I confirmed that the test may have tested
VACCUM before slot.xmin was advanced. Therefore, to improve the test, I
modified test to wait for the publisher's request message appearing twice, as
after the fix, the apply worker should keep waiting for publisher status until the
prepared txn is committed.Also, to reduce test time, I moved the test into the existing 035 test.
Here is the updated test.
I noticed a BF failure[1]https://buildfarm.postgresql.org/cgi-bin/show_stage_log.pl?nm=scorpion&dt=2025-09-11%2001%3A17%3A25&stg=subscription-check on this test. The log shows that the apply worker
advances the non-removable xid to the latest state before waiting for the
prepared transaction to commit. Upon reviewing the log, I didn't find any clues
of a bug in the code. One potential explanation is that the prepared transaction
hasn't reached the injection point before the apply worker requests the
publisher status.
The log lacks the timing for when the injection point is triggered and only
includes:
pub: 2025-09-11 03:40:05.667 CEST [396867][client backend][8/3:0] LOG: statement: COMMIT PREPARED 'txn_with_later_commit_ts';
..
sub: 2025-09-11 03:40:05.684 CEST [396798][logical replication apply worker][16/0:0] DEBUG: sending publisher status request message
Although the statement on the publisher appears before the publisher request,
the statement log is generated prior to command execution. Thus, it's possible
the injection point is triggered after responding to the publisher status.
After checking some other tap tests using injection points, most of them ensure
the injection is triggered before proceeding with the test (by waiting for the
wait event of injection point). We could also add this in the test:
$node_B->wait_for_event('client backend', 'commit-after-delay-checkpoint');
Here is a small patch.
Best Regards,
Hou zj
Attachments:
v1-0001-Fix-unstable-test-in-6456c6e.patchapplication/octet-stream; name=v1-0001-Fix-unstable-test-in-6456c6e.patchDownload
From ae7940362d10df42dcfa78bd31c5117a67ab088a Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Thu, 11 Sep 2025 16:29:49 +0800
Subject: [PATCH v1] Fix unstable test in 6456c6e
The test introduced in commit 6456c6e anticipates a backend to execute COMMIT
PREPARED and enter the injection point 'commit-after-delay-checkpoint' within
the commit critical section. Consequently, the apply worker on the subscriber
should wait for this transaction to finish. However, as the test does not ensure
the injection point is triggered, there exists a window where the apply
worker might proceed prematurely during the execution of COMMIT PREPARED. This
commit addresses the issue by ensuring the wait event for the injection point is
triggered before conducting further tests.
---
src/test/subscription/t/035_conflicts.pl | 3 +++
1 file changed, 3 insertions(+)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index db0d5b464e8..880551fc69d 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -475,6 +475,9 @@ if ($injection_points_supported != 0)
}
);
+ # Wait until the backend enters the injection point
+ $node_B->wait_for_event('client backend', 'commit-after-delay-checkpoint');
+
# Confirm the update is suspended
$result =
$node_B->safe_psql('postgres', 'SELECT * FROM tab WHERE a = 1');
--
2.51.0.windows.1
On Thu, Sep 11, 2025 at 2:29 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Monday, September 8, 2025 7:21 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Monday, September 8, 2025 3:13 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:On Fri, Sep 5, 2025 at 5:03 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:Here are v2 patches which addressed above comments.
I have pushed the first patch. I find that the test can't reliably fail without a fix.
Can you please investigate it?Thank you for catching this issue. I confirmed that the test may have tested
VACCUM before slot.xmin was advanced. Therefore, to improve the test, I
modified test to wait for the publisher's request message appearing twice, as
after the fix, the apply worker should keep waiting for publisher status until the
prepared txn is committed.Also, to reduce test time, I moved the test into the existing 035 test.
Here is the updated test.
I noticed a BF failure[1] on this test. The log shows that the apply worker
advances the non-removable xid to the latest state before waiting for the
prepared transaction to commit. Upon reviewing the log, I didn't find any clues
of a bug in the code. One potential explanation is that the prepared transaction
hasn't reached the injection point before the apply worker requests the
publisher status.The log lacks the timing for when the injection point is triggered and only
includes:pub: 2025-09-11 03:40:05.667 CEST [396867][client backend][8/3:0] LOG: statement: COMMIT PREPARED 'txn_with_later_commit_ts';
..
sub: 2025-09-11 03:40:05.684 CEST [396798][logical replication apply worker][16/0:0] DEBUG: sending publisher status request messageAlthough the statement on the publisher appears before the publisher request,
the statement log is generated prior to command execution. Thus, it's possible
the injection point is triggered after responding to the publisher status.After checking some other tap tests using injection points, most of them ensure
the injection is triggered before proceeding with the test (by waiting for the
wait event of injection point). We could also add this in the test:$node_B->wait_for_event('client backend', 'commit-after-delay-checkpoint');
Here is a small patch.
Agree with the analysis. The patch looks good.
thanks
Shveta
On Wed, Sep 10, 2025 at 2:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Sep 10, 2025 at 9:08 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, September 9, 2025 7:01 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Few other comments: 1. + /* + * Return if the launcher has not initialized oldest_nonremovable_xid. + * + * It might seem feasible to directly check the conflict detection + * slot.xmin instead of relying on the launcher to assign the worker's + * oldest_nonremovable_xid; however, that could lead to a race + condition + * where slot.xmin is set to InvalidTransactionId immediately after the + * check. In such cases, oldest_nonremovable_xid would no longer be + * protected by a replication slot and could become unreliable if a + * wraparound occurs. + */ + if (!TransactionIdIsValid(nonremovable_xid)) + return;I understand the reason why we get assigned the worker's non_removable_xid
from launcher but all this makes the code complex to understand. Isn't there
any other way to achieve it? One naïve and inefficient way could be to just
restart the worker probably after updating its retentionactive flag. I am not
suggesting to make this change but just a brainstorming point.I'll keep thinking about it, and for now, I've added a comment mentioning
that rebooting is a simpler solution.Say we have a LW LogicalRepRetentionLock. We acquire it in SHARED mode
as soon as we encounter the first subscription with retain_dead_tuples
set during the traversal of the sublist. We release it only after
updating xmin outside the sublist traversal. We then acquire it in
EXCLUSIVE mode to fetch the resume the retention in worker for the
period till we fetch slot's xmin.This will close the above race condition but acquiring LWLock while
traversing subscription is not advisable as that will make it
uninterruptible. The other possibility is to use some heavy-weight
lock here, say a lock on pg_subscription catalog but that has a
drawback that it can conflict with DDL operations. Yet another way is
to invent a new lock-type for this.OTOH, the simple strategy to let apply-worker restart to resume
retention will keep the handling simpler. We do something similar at
the start of apply-worker if we find that some subscription parameter
is changed. I think we need more opinions on this matter.
IMHO the situation of retention being disabled and re-enabled is not a
common occurrence. It typically happens in specific scenarios where
there's a significant replication lag or the user has not configured
the retention timeout correctly. Because these are corner cases, I
believe we should avoid over-engineering a solution and simply restart
the worker, as Amit suggested.
--
Regards,
Dilip Kumar
Google
On Thu, Sep 11, 2025 at 3:54 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Sep 10, 2025 at 2:09 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Sep 10, 2025 at 9:08 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, September 9, 2025 7:01 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Few other comments: 1. + /* + * Return if the launcher has not initialized oldest_nonremovable_xid. + * + * It might seem feasible to directly check the conflict detection + * slot.xmin instead of relying on the launcher to assign the worker's + * oldest_nonremovable_xid; however, that could lead to a race + condition + * where slot.xmin is set to InvalidTransactionId immediately after the + * check. In such cases, oldest_nonremovable_xid would no longer be + * protected by a replication slot and could become unreliable if a + * wraparound occurs. + */ + if (!TransactionIdIsValid(nonremovable_xid)) + return;I understand the reason why we get assigned the worker's non_removable_xid
from launcher but all this makes the code complex to understand. Isn't there
any other way to achieve it? One naïve and inefficient way could be to just
restart the worker probably after updating its retentionactive flag. I am not
suggesting to make this change but just a brainstorming point.I'll keep thinking about it, and for now, I've added a comment mentioning
that rebooting is a simpler solution.Say we have a LW LogicalRepRetentionLock. We acquire it in SHARED mode
as soon as we encounter the first subscription with retain_dead_tuples
set during the traversal of the sublist. We release it only after
updating xmin outside the sublist traversal. We then acquire it in
EXCLUSIVE mode to fetch the resume the retention in worker for the
period till we fetch slot's xmin.This will close the above race condition but acquiring LWLock while
traversing subscription is not advisable as that will make it
uninterruptible. The other possibility is to use some heavy-weight
lock here, say a lock on pg_subscription catalog but that has a
drawback that it can conflict with DDL operations. Yet another way is
to invent a new lock-type for this.OTOH, the simple strategy to let apply-worker restart to resume
retention will keep the handling simpler. We do something similar at
the start of apply-worker if we find that some subscription parameter
is changed. I think we need more opinions on this matter.IMHO the situation of retention being disabled and re-enabled is not a
common occurrence. It typically happens in specific scenarios where
there's a significant replication lag or the user has not configured
the retention timeout correctly. Because these are corner cases, I
believe we should avoid over-engineering a solution and simply restart
the worker, as Amit suggested.
+1
While it's ideal if workers could initialize their
oldest_nonremovable_xid values on-the-fly, I believe we can begin with
the simple solution given that stopping and resuming retaining of
conflict info would not happen so often. In fact, frequent stops and
restarts would typically be a sign that users might be not configuring
the options properly for their systems. IIUC if the workers are able
to do that, we can support to activate retain_conflict_info even for
enabled subscriptions. I think we can leave it for future improvements
if necessary.
Regards,
--
Masahiko Sawada
Amazon Web Services: https://aws.amazon.com
On Friday, September 12, 2025 2:39 AM Masahiko Sawada <sawada.mshk@gmail.com> wrote:
On Thu, Sep 11, 2025 at 3:54 AM Dilip Kumar <dilipbalaut@gmail.com> wrote:
On Wed, Sep 10, 2025 at 2:09 PM Amit Kapila <amit.kapila16@gmail.com>
wrote:
On Wed, Sep 10, 2025 at 9:08 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:On Tuesday, September 9, 2025 7:01 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:
On Tue, Sep 9, 2025 at 11:47 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com>
wrote:Few other comments: 1. + /* + * Return if the launcher has not initialized oldest_nonremovable_xid. + * + * It might seem feasible to directly check the conflict + detection + * slot.xmin instead of relying on the launcher to assign the + worker's + * oldest_nonremovable_xid; however, that could lead to a race + condition + * where slot.xmin is set to InvalidTransactionId immediately + after the + * check. In such cases, oldest_nonremovable_xid would no + longer be + * protected by a replication slot and could become unreliable + if a + * wraparound occurs. + */ + if (!TransactionIdIsValid(nonremovable_xid)) + return;I understand the reason why we get assigned the worker's
non_removable_xid from launcher but all this makes the code
complex to understand. Isn't there any other way to achieve it?
One naïve and inefficient way could be to just restart the
worker probably after updating its retentionactive flag. I am notsuggesting to make this change but just a brainstorming point.
I'll keep thinking about it, and for now, I've added a comment
mentioning that rebooting is a simpler solution.Say we have a LW LogicalRepRetentionLock. We acquire it in SHARED
mode as soon as we encounter the first subscription with
retain_dead_tuples set during the traversal of the sublist. We
release it only after updating xmin outside the sublist traversal.
We then acquire it in EXCLUSIVE mode to fetch the resume the
retention in worker for the period till we fetch slot's xmin.This will close the above race condition but acquiring LWLock while
traversing subscription is not advisable as that will make it
uninterruptible. The other possibility is to use some heavy-weight
lock here, say a lock on pg_subscription catalog but that has a
drawback that it can conflict with DDL operations. Yet another way
is to invent a new lock-type for this.OTOH, the simple strategy to let apply-worker restart to resume
retention will keep the handling simpler. We do something similar at
the start of apply-worker if we find that some subscription
parameter is changed. I think we need more opinions on this matter.IMHO the situation of retention being disabled and re-enabled is not a
common occurrence. It typically happens in specific scenarios where
there's a significant replication lag or the user has not configured
the retention timeout correctly. Because these are corner cases, I
believe we should avoid over-engineering a solution and simply restart
the worker, as Amit suggested.+1
While it's ideal if workers could initialize their oldest_nonremovable_xid values
on-the-fly, I believe we can begin with the simple solution given that stopping
and resuming retaining of conflict info would not happen so often. In fact,
frequent stops and restarts would typically be a sign that users might be not
configuring the options properly for their systems. IIUC if the workers are able to
do that, we can support to activate retain_conflict_info even for enabled
subscriptions. I think we can leave it for future improvements if necessary.
I agree. Here is a V73 patch that will restart the worker if the retention
resumes. I also addressed other comments posted by Amit[1]/messages/by-id/CAA4eK1LA7mnvKT9L8Nx_h+0TCvq-Ob2BGPO1bQKhkGHtoZKsow@mail.gmail.com.
[1]: /messages/by-id/CAA4eK1LA7mnvKT9L8Nx_h+0TCvq-Ob2BGPO1bQKhkGHtoZKsow@mail.gmail.com
Best Regards,
Hou zj
Attachments:
v73-0001-Allow-conflict-relevant-data-retention-to-resume.patchapplication/octet-stream; name=v73-0001-Allow-conflict-relevant-data-retention-to-resume.patchDownload
From 04a3b4c8f3f308228ff3f994b3d3b32e02d8332d Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 9 Sep 2025 11:03:36 +0800
Subject: [PATCH v73] Allow conflict-relevant data retention to resume
This commit enables automatic recovery of conflict-relevant data retention for a
subscription. If the retention duration for a subscription previously exceeded
the max_retention_duration and caused retention to stop, the retention can
resume once the duration falls within the acceptable limits.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/replication/logical/worker.c | 235 ++++++++++++++++++----
src/test/subscription/t/035_conflicts.pl | 27 +++
3 files changed, 227 insertions(+), 44 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ed82cf1809e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will automatically resume when at least one
+ apply worker confirms that the retention duration is within the
+ specified limit, or when a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>. Alternatively, retention
+ can be manually resumed by re-enabling <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ee6ac22329f..7498afc92c0 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,17 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and the worker will be restarted. Note that the state could
+ * transition to RDT_RESUME_CONFLICT_INFO_RETENTION at any phase if the
+ * retention has been stopped, but max_retention_duration is now set to 0.
+ *
* 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) ->
@@ -381,7 +392,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION
} RetainDeadTuplesPhase;
/*
@@ -568,10 +580,16 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool should_resume_retention_immediately(RetainDeadTuplesData *rdt_data,
+ bool status_received);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
+static void apply_worker_exit(void);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -4345,6 +4363,13 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+
process_rdt_phase_transition(rdt_data, status_received);
}
@@ -4367,10 +4392,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4399,6 +4420,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4522,7 +4546,20 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
* retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase to
+ * retry all phases. This is required to recalculate the current wait
+ * time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ else
+ reset_retention_data_fields(rdt_data);
+
return;
+ }
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4643,7 +4680,20 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase to
+ * retry all phases. This is required to recalculate the current wait
+ * time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ else
+ reset_retention_data_fields(rdt_data);
+
return;
+ }
/*
* Update and check the remote flush position if we are applying changes
@@ -4672,6 +4722,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * Reaching this point implies should_stop_conflict_info_retention()
+ * returned false earlier, indicating that the most recent duration for
+ * advancing the non-removable transaction ID is within the
+ * max_retention_duration.
+ *
+ * Therefore, if conflict info retention was previously stopped due to a
+ * timeout, proceed to resume retention now.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ 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
@@ -4701,10 +4766,6 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* If retention should be stopped, transition to the
* RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
* false.
- *
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4735,11 +4796,6 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
-
return true;
}
@@ -4748,6 +4804,107 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Return if unable to update subretentionactive (see
+ * update_retention_status).
+ */
+ if (!update_retention_status(false))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Check whether retention should be resumed immediately if it has been
+ * previously stopped, but max_retention_duration is now set to 0.
+ */
+static bool
+should_resume_retention_immediately(RetainDeadTuplesData *rdt_data, bool status_received)
+{
+ /* Return false if retention is already being resumed */
+ if (rdt_data->phase == RDT_RESUME_CONFLICT_INFO_RETENTION)
+ return false;
+
+ /* Return false if max_retention_duration is not 0 */
+ if (MySubscription->maxretention)
+ return false;
+
+ /*
+ * Do not resume when waiting for publisher status, as doing so may result
+ * in the message being processed after the data and phase have been
+ * reset, potentially causing it to be mistakenly identified as a new
+ * message. This could lead to the premature advancement of
+ * oldest_nonremovable_xid.
+ */
+ if (rdt_data->phase == RDT_WAIT_FOR_PUBLISHER_STATUS &&
+ !status_received)
+ return false;
+
+ /*
+ * Resume retention if we are in the process of stopping or have already
+ * stopped retention.
+ */
+ return rdt_data->phase == RDT_STOP_CONFLICT_INFO_RETENTION ||
+ !MySubscription->retentionactive;
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Return if unable to update subretentionactive (see
+ * update_retention_status).
+ */
+ if (!update_retention_status(true))
+ return;
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+
+ /*
+ * Restart the worker to allow the launcher to initialize
+ * oldest_nonremovable_xid value at startup.
+ *
+ * An alternative approach is using the conflict detection slot.xmin to
+ * initialize the oldest_nonremovable_xid on-the-fly, without restarting
+ * the worker. However, this could create a race condition where the
+ * launcher invalidates slot.xmin immediately after the worker resumes
+ * retention, making oldest_nonremovable_xid unreliable if xid wraparound
+ * occurs. While implementing a heavy lock to prevent concurrent slot
+ * updates by the launcher is feasible, given that resuming is an
+ * infrequent operation, it may not be worthwhile to handle it.
+ */
+ apply_worker_exit();
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Returns true upon successful update; however, if currently within an active
+ * transaction, skip the update and return false.
+ */
+static bool
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4755,7 +4912,7 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
* rollback of catalog updates if the application fails subsequently.
*/
if (IsTransactionState())
- return;
+ return false;
StartTransactionCommand();
@@ -4765,26 +4922,18 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
/* Notify launcher to update the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
+
+ return true;
}
/*
@@ -4809,19 +4958,20 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
- * If there is no activity on the node, we progressively double the interval
- * used to advance non-removable transaction ID. This helps conserve CPU
- * and network resources when there's little benefit to frequent updates.
+ * If there is no activity on the node or retention has been stopped, we
+ * progressively double the interval used to advance non-removable transaction
+ * ID. This helps conserve CPU and network resources when there's little benefit
+ * to frequent updates.
*
* The interval is capped by the lowest of the following:
* - wal_receiver_status_interval (if set),
* - a default maximum of 3 minutes,
- * - max_retention_duration.
+ * - max_retention_duration (if retention is active).
*
- * This ensures the interval never exceeds the retention boundary, even if
- * other limits are higher. Once activity resumes on the node, the interval
- * is reset to lesser of 100ms and max_retention_duration, allowing timely
- * advancement of non-removable transaction ID.
+ * This ensures the interval never exceeds the retention boundary, even if other
+ * limits are higher. Once activity resumes on the node and the retention is
+ * active, the interval is reset to lesser of 100ms and max_retention_duration,
+ * allowing timely advancement of non-removable transaction ID.
*
* 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.
@@ -4829,7 +4979,8 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
static void
adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
{
- if (!new_xid_found && rdt_data->xid_advance_interval)
+ if (rdt_data->xid_advance_interval &&
+ (!new_xid_found || !MySubscription->retentionactive))
{
int max_interval = wal_receiver_status_interval
? wal_receiver_status_interval * 1000
@@ -4851,9 +5002,13 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
- /* Ensure the wait time remains within the maximum limit */
- rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
- MySubscription->maxretention);
+ /*
+ * Ensure the wait time remains within the maximum limit when retention is
+ * active.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 880551fc69d..6f371b4b1f4 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -631,6 +631,33 @@ $node_B->safe_psql('postgres',
$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
$node_B->reload;
+###############################################################################
+# Check that dead tuple retention resumes when the max_retention_duration is set
+# 0.
+###############################################################################
+
+$log_offset = -s $node_A->logfile;
+
+# Set max_retention_duration to 0
+$node_A->safe_psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 0);");
+
+# Confirm that the retention resumes
+$node_A->wait_for_log(
+ qr/logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
+.*DETAIL:.* Retention of information used for conflict detection is now indefinite.*/,
+ $log_offset);
+
+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");
+
+$result = $node_A->safe_psql('postgres',
+ "SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'retention is active');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
On Fri, Sep 12, 2025 at 8:55 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I agree. Here is a V73 patch that will restart the worker if the retention
resumes. I also addressed other comments posted by Amit[1].
Few comments:
============
* In adjust_xid_advance_interval(), for the case when retention is not
active, we still cap the interval by wal_receiver_status_interval. Is
that required or do we let it go till 3 minutes? We can add a new else
if loop to keep the code clear, if you agree with this.
*
+ /*
+ * Resume retention immediately if required. (See
+ * should_resume_retention_immediately() for details).
+ */
+ if (should_resume_retention_immediately(rdt_data, status_received))
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
Is this optimization for the case when we are in stop_phase or after
stop_phase and someone has changed maxretention to 0? If so, I don't
think it is worth optimizing for such a rare case at the cost of
making code difficult to understand.
Apart from this, I have changed a few comments in the attached.
--
With Regards,
Amit Kapila.
Attachments:
v73-0001-amit.patch.txttext/plain; charset=US-ASCII; name=v73-0001-amit.patch.txtDownload
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 1274192de7e..7d1ed506622 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4363,10 +4363,7 @@ maybe_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data,
if (!can_advance_nonremovable_xid(rdt_data))
return;
- /*
- * Resume retention immediately if required. (See
- * should_resume_retention_immediately() for details).
- */
+ /* Resume dead_tuples retention immediately if required. */
if (should_resume_retention_immediately(rdt_data, status_received))
rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
@@ -4549,8 +4546,8 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
{
/*
* Stop retention if not yet. Otherwise, reset to the initial phase to
- * retry all phases. This is required to recalculate the current wait
- * time and resume retention if the time falls within
+ * retry resuming retention. This reset is required to recalculate the
+ * current wait time and resume retention if the time falls within
* max_retention_duration.
*/
if (MySubscription->retentionactive)
@@ -4683,8 +4680,8 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
{
/*
* Stop retention if not yet. Otherwise, reset to the initial phase to
- * retry all phases. This is required to recalculate the current wait
- * time and resume retention if the time falls within
+ * retry resuming retention. This reset is required to recalculate the
+ * current wait time and resume retention if the time falls within
* max_retention_duration.
*/
if (MySubscription->retentionactive)
@@ -4763,9 +4760,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* Check whether conflict information retention should be stopped due to
* exceeding the maximum wait time (max_retention_duration).
*
- * If retention should be stopped, transition to the
- * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
- * false.
+ * If retention should be stopped, return true. Otherwise, return false.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
On Fri, Sep 12, 2025 at 8:55 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
I agree. Here is a V73 patch that will restart the worker if the retention
resumes. I also addressed other comments posted by Amit[1].
Thanks for the patch. Few comments:
1)
There is a small window where worker can exit while resuming
retention and launcher can end up acessign stale worker info.
Lets say launcher is at a stage where it has fetched worker:
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
And after this point, before the launcher could do
compute_min_nonremovable_xid(), the worker has stopped retention and
resumed as well. Now the worker has exited but in
compute_min_nonremovable_xid(), launcher will still use the
worker-info fetched previously.
2)
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase to
+ * retry all phases. This is required to recalculate the current wait
+ * time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ else
+ reset_retention_data_fields(rdt_data);
+
return;
+ }
Shall we have an Assert( !MyLogicalRepWorker->oldest_nonremovable_xid)
in 'else' part above?
thanks
Shveta
On Friday, September 12, 2025 4:48 PM shveta malik <shveta.malik@gmail.com> wrote:
On Fri, Sep 12, 2025 at 8:55 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:I agree. Here is a V73 patch that will restart the worker if the
retention resumes. I also addressed other comments posted by Amit[1].Thanks for the patch. Few comments:
Thanks for the comments!
1)
There is a small window where worker can exit while resuming retention and
launcher can end up acessign stale worker info.Lets say launcher is at a stage where it has fetched worker:
w = logicalrep_worker_find(sub->oid, InvalidOid, false);And after this point, before the launcher could do
compute_min_nonremovable_xid(), the worker has stopped retention and
resumed as well. Now the worker has exited but in
compute_min_nonremovable_xid(), launcher will still use the worker-info
fetched previously.
Thanks for catching this, I have fixed by computing the xid under
LogicalRepWorkerLock.
2)
if (should_stop_conflict_info_retention(rdt_data)) + { + /* + * Stop retention if not yet. Otherwise, reset to the initial phase +to + * retry all phases. This is required to recalculate the current +wait + * time and resume retention if the time falls within + * max_retention_duration. + */ + if (MySubscription->retentionactive) + rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION; + else + reset_retention_data_fields(rdt_data); + return; + }Shall we have an Assert( !MyLogicalRepWorker->oldest_nonremovable_xid)
in 'else' part above?
Added.
Here is the V74 patch which addressed all comments.
Best Regards,
Hou zj
Attachments:
v74-0001-Allow-conflict-relevant-data-retention-to-resume.patchapplication/octet-stream; name=v74-0001-Allow-conflict-relevant-data-retention-to-resume.patchDownload
From f28b20706ea68e63d05029949a49f09e404e2751 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 9 Sep 2025 11:03:36 +0800
Subject: [PATCH v74] Allow conflict-relevant data retention to resume
This commit enables automatic recovery of conflict-relevant data retention for a
subscription. If the retention duration for a subscription previously exceeded
the max_retention_duration and caused retention to stop, the retention can
resume once the duration falls within the acceptable limits.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/replication/logical/launcher.c | 10 +-
src/backend/replication/logical/worker.c | 217 ++++++++++++++++-----
src/test/subscription/t/035_conflicts.pl | 27 +++
4 files changed, 213 insertions(+), 50 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ed82cf1809e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will automatically resume when at least one
+ apply worker confirms that the retention duration is within the
+ specified limit, or when a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>. Alternatively, retention
+ can be manually resumed by re-enabling <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add2e2e066c..1d42ca91ea3 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1261,24 +1261,30 @@ ApplyLauncherMain(Datum main_arg)
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
- LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
{
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers.
+ * workers. This computation is performed under
+ * LogicalRepWorkerLock to avoid accessing invalid worker
+ * information in scenarios where a worker may exit and reset
+ * data concurrently.
*/
if (sub->retaindeadtuples &&
sub->retentionactive &&
can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
+ LWLockRelease(LogicalRepWorkerLock);
+
/* worker is running already */
continue;
}
+ LWLockRelease(LogicalRepWorkerLock);
+
/*
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions actively retaining dead tuples
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ee6ac22329f..6d9ff6828a4 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,15 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and the worker will be restarted.
+ *
* 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) ->
@@ -381,7 +390,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -568,10 +578,14 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
+static void apply_worker_exit(void);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -4367,10 +4381,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4399,6 +4409,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4522,7 +4535,25 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
* retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase to
+ * retry resuming retention. This reset is required to recalculate the
+ * current wait time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ }
+ else
+ {
+ Assert(!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid));
+ reset_retention_data_fields(rdt_data);
+ }
+
return;
+ }
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4643,7 +4674,25 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase to
+ * retry resuming retention. This reset is required to recalculate the
+ * current wait time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ if (MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ }
+ else
+ {
+ Assert(!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid));
+ reset_retention_data_fields(rdt_data);
+ }
+
return;
+ }
/*
* Update and check the remote flush position if we are applying changes
@@ -4672,6 +4721,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * Reaching this point implies should_stop_conflict_info_retention()
+ * returned false earlier, indicating that the most recent duration for
+ * advancing the non-removable transaction ID is within the
+ * max_retention_duration or max_retention_duration is set to 0.
+ *
+ * Therefore, if conflict info retention was previously stopped due to a
+ * timeout, proceed to resume retention now.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ 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
@@ -4698,13 +4762,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* Check whether conflict information retention should be stopped due to
* exceeding the maximum wait time (max_retention_duration).
*
- * If retention should be stopped, transition to the
- * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
- * false.
- *
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * If retention should be stopped, return true. Otherwise, return false.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4735,11 +4793,6 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
-
return true;
}
@@ -4748,6 +4801,73 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Return if unable to update subretentionactive (see
+ * update_retention_status).
+ */
+ if (!update_retention_status(false))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /*
+ * Return if unable to update subretentionactive (see
+ * update_retention_status).
+ */
+ if (!update_retention_status(true))
+ return;
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is now indefinite."));
+
+ /*
+ * Restart the worker to allow the launcher to initialize
+ * oldest_nonremovable_xid value at startup.
+ *
+ * An alternative approach is using the conflict detection slot.xmin to
+ * initialize the oldest_nonremovable_xid on-the-fly, without restarting
+ * the worker. However, this could create a race condition where the
+ * launcher invalidates slot.xmin immediately after the worker resumes
+ * retention, making oldest_nonremovable_xid unreliable if xid wraparound
+ * occurs. While implementing a heavy lock to prevent concurrent slot
+ * updates by the launcher is feasible, given that resuming is an
+ * infrequent operation, it may not be worthwhile to handle it.
+ */
+ apply_worker_exit();
+}
+
+/*
+ * Update pg_subscription.subretentionactive to the given value within a new
+ * transaction.
+ *
+ * Returns true upon successful update; however, if currently within an active
+ * transaction, skip the update and return false.
+ */
+static bool
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4755,7 +4875,7 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
* rollback of catalog updates if the application fails subsequently.
*/
if (IsTransactionState())
- return;
+ return false;
StartTransactionCommand();
@@ -4765,26 +4885,18 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
/* Notify launcher to update the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
+
+ return true;
}
/*
@@ -4809,19 +4921,20 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
- * If there is no activity on the node, we progressively double the interval
- * used to advance non-removable transaction ID. This helps conserve CPU
- * and network resources when there's little benefit to frequent updates.
+ * If there is no activity on the node or retention has been stopped, we
+ * progressively double the interval used to advance non-removable transaction
+ * ID. This helps conserve CPU and network resources when there's little benefit
+ * to frequent updates.
*
* The interval is capped by the lowest of the following:
- * - wal_receiver_status_interval (if set),
+ * - wal_receiver_status_interval (if set and retention is active),
* - a default maximum of 3 minutes,
- * - max_retention_duration.
+ * - max_retention_duration (if retention is active).
*
- * This ensures the interval never exceeds the retention boundary, even if
- * other limits are higher. Once activity resumes on the node, the interval
- * is reset to lesser of 100ms and max_retention_duration, allowing timely
- * advancement of non-removable transaction ID.
+ * This ensures the interval never exceeds the retention boundary, even if other
+ * limits are higher. Once activity resumes on the node and the retention is
+ * active, the interval is reset to lesser of 100ms and max_retention_duration,
+ * allowing timely advancement of non-removable transaction ID.
*
* 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.
@@ -4829,7 +4942,7 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
static void
adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
{
- if (!new_xid_found && rdt_data->xid_advance_interval)
+ if (rdt_data->xid_advance_interval && !new_xid_found)
{
int max_interval = wal_receiver_status_interval
? wal_receiver_status_interval * 1000
@@ -4842,6 +4955,18 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
}
+ else if (rdt_data->xid_advance_interval &&
+ !MySubscription->retentionactive)
+ {
+ /*
+ * Retention has been stopped, so double the interval, but not beyond
+ * 3 minutes. The wal_receiver_status_interval is not considered as a
+ * maximum, since the chance of retention resuming is less than that
+ * of activity resuming.
+ */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
+ MAX_XID_ADVANCE_INTERVAL);
+ }
else
{
/*
@@ -4851,9 +4976,13 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
- /* Ensure the wait time remains within the maximum limit */
- rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
- MySubscription->maxretention);
+ /*
+ * Ensure the wait time remains within the maximum limit when retention is
+ * active.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 880551fc69d..6f371b4b1f4 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -631,6 +631,33 @@ $node_B->safe_psql('postgres',
$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
$node_B->reload;
+###############################################################################
+# Check that dead tuple retention resumes when the max_retention_duration is set
+# 0.
+###############################################################################
+
+$log_offset = -s $node_A->logfile;
+
+# Set max_retention_duration to 0
+$node_A->safe_psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 0);");
+
+# Confirm that the retention resumes
+$node_A->wait_for_log(
+ qr/logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
+.*DETAIL:.* Retention of information used for conflict detection is now indefinite.*/,
+ $log_offset);
+
+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");
+
+$result = $node_A->safe_psql('postgres',
+ "SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'retention is active');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
On Friday, September 12, 2025 4:43 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 12, 2025 at 8:55 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:I agree. Here is a V73 patch that will restart the worker if the
retention resumes. I also addressed other comments posted by Amit[1].Few comments:
============
* In adjust_xid_advance_interval(), for the case when retention is not active,
we still cap the interval by wal_receiver_status_interval. Is that required or do
we let it go till 3 minutes? We can add a new else if loop to keep the code clear,
if you agree with this.
I agree we can let it go till 3 mins, and changed the patch as suggested.
* + /* + * Resume retention immediately if required. (See + * should_resume_retention_immediately() for details). + */ + if (should_resume_retention_immediately(rdt_data, status_received)) + rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;Is this optimization for the case when we are in stop_phase or after stop_phase
and someone has changed maxretention to 0? If so, I don't think it is worth
optimizing for such a rare case at the cost of making code difficult to
understand.
Agreed. I removed this in V74.
Apart from this, I have changed a few comments in the attached.
Thanks for the patch, it looks good to me. I have merged it in V74.
Best Regards,
Hou zj
On Fri, Sep 12, 2025 at 3:39 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Here is the V74 patch which addressed all comments.
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will
resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention of information used for conflict detection is
now within the max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention of information used for conflict detection is
now indefinite."));
The detail message doesn't seems to convey the correct meaning as the
duration is compared with something vague. How about changing
errdetail messages as follows:
"Retention is re-enabled as the apply process is advancing its xmin
within the configured max_retention_duration of %u ms."
"Retention is re-enabled as max_retention_duration is set to unlimited."
If you agree with the above then we can consider changing the existing
errdetail related to stop_retention functionality as follows:
"Retention is stopped as the apply process is not advancing its xmin
within the configured max_retention_duration of %u ms."
Apart from these, I have made some cosmetic changes in the attached.
--
With Regards,
Amit Kapila.
Attachments:
v74_amit_1.txttext/plain; charset=US-ASCII; name=v74_amit_1.txtDownload
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index 1d42ca91ea3..ed6ba5b2d7e 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1267,10 +1267,10 @@ ApplyLauncherMain(Datum main_arg)
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers. This computation is performed under
- * LogicalRepWorkerLock to avoid accessing invalid worker
- * information in scenarios where a worker may exit and reset
- * data concurrently.
+ * workers. This computation is performed while holding
+ * LogicalRepWorkerLock to prevent accessing invalid worker
+ * data, in scenarios where a worker might exit and reset
+ * its state concurrently.
*/
if (sub->retaindeadtuples &&
sub->retentionactive &&
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index 6d9ff6828a4..722a3678e7f 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -4723,12 +4723,12 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
/*
* Reaching this point implies should_stop_conflict_info_retention()
- * returned false earlier, indicating that the most recent duration for
+ * returned false earlier, meaning that the most recent duration for
* advancing the non-removable transaction ID is within the
* max_retention_duration or max_retention_duration is set to 0.
*
* Therefore, if conflict info retention was previously stopped due to a
- * timeout, proceed to resume retention now.
+ * timeout, it is now safe to resume retention.
*/
if (!MySubscription->retentionactive)
{
@@ -4803,8 +4803,9 @@ static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
{
/*
- * Return if unable to update subretentionactive (see
- * update_retention_status).
+ * If the retention status cannot be updated (e.g., due to active
+ * transaction), skip further processing to avoid inconsistent retention
+ * behavior.
*/
if (!update_retention_status(false))
return;
@@ -4828,10 +4829,7 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
static void
resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
{
- /*
- * Return if unable to update subretentionactive (see
- * update_retention_status).
- */
+ /* We can't resume retention without updating retention status. */
if (!update_retention_status(true))
return;
@@ -4844,27 +4842,30 @@ resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
: errdetail("Retention of information used for conflict detection is now indefinite."));
/*
- * Restart the worker to allow the launcher to initialize
- * oldest_nonremovable_xid value at startup.
+ * Restart the worker to let the launcher initialize
+ * oldest_nonremovable_xid at startup.
+ *
+ * While it's technically possible to derive this value on-the-fly using
+ * the conflict detection slot's xmin, doing so risks a race condition: the
+ * launcher might clean slot.xmin just after retention resumes. This would
+ * make oldest_nonremovable_xid unreliable, especially during xid
+ * wraparound.
*
- * An alternative approach is using the conflict detection slot.xmin to
- * initialize the oldest_nonremovable_xid on-the-fly, without restarting
- * the worker. However, this could create a race condition where the
- * launcher invalidates slot.xmin immediately after the worker resumes
- * retention, making oldest_nonremovable_xid unreliable if xid wraparound
- * occurs. While implementing a heavy lock to prevent concurrent slot
- * updates by the launcher is feasible, given that resuming is an
- * infrequent operation, it may not be worthwhile to handle it.
+ * Although this can be prevented by introducing heavy weight locking, the
+ * complexity it will bring doesn't seem worthwhile given how rarely
+ * retention is resumed.
*/
apply_worker_exit();
}
/*
- * Update pg_subscription.subretentionactive to the given value within a new
- * transaction.
+ * Updates pg_subscription.subretentionactive to the given value within a
+ * new transaction.
+ *
+ * If already inside an active transaction, skips the update and returns
+ * false.
*
- * Returns true upon successful update; however, if currently within an active
- * transaction, skip the update and return false.
+ * Returns true if the update is successfully performed.
*/
static bool
update_retention_status(bool active)
@@ -4959,10 +4960,10 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
!MySubscription->retentionactive)
{
/*
- * Retention has been stopped, so double the interval, but not beyond
- * 3 minutes. The wal_receiver_status_interval is not considered as a
- * maximum, since the chance of retention resuming is less than that
- * of activity resuming.
+ * Retention has been stopped, so double the interval-capped at a
+ * maximum of 3 minutes. The wal_receiver_status_interval is
+ * intentionally not used as a upper bound, since the likelihood of
+ * retention resuming is lower than that of general activity resuming.
*/
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
MAX_XID_ADVANCE_INTERVAL);
@@ -4977,8 +4978,8 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
}
/*
- * Ensure the wait time remains within the maximum limit when retention is
- * active.
+ * Ensure the wait time remains within the maximum retention time limit
+ * when retention is active.
*/
if (MySubscription->retentionactive)
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
One concern:
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ /*
+ * Stop retention if not yet. Otherwise, reset to the initial phase to
+ * retry all phases. This is required to recalculate the current wait
+ * time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
+ else
+ reset_retention_data_fields(rdt_data);
+
return;
+ }
Instead of above code changes, shall we have:
if (should_stop_conflict_info_retention(rdt_data))
rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION; (always)
And then stop_conflict_info_retention() should have these checks:
if (MySubscription->retentionactive)
{
...update flag and perform stop (current functionality)
}
else
{
Assert(!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid));
reset_retention_data_fields(rdt_data);
}
thanks
Shveta
On Monday, September 15, 2025 12:55 PM shveta malik <shveta.malik@gmail.com> wrote:
One concern:
if (should_stop_conflict_info_retention(rdt_data)) + { + /* + * Stop retention if not yet. Otherwise, reset to the initial phase to + * retry all phases. This is required to recalculate the current wait + * time and resume retention if the time falls within + * max_retention_duration. + */ + if (MySubscription->retentionactive) + rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION; + else + reset_retention_data_fields(rdt_data); + return; + }Instead of above code changes, shall we have:
if (should_stop_conflict_info_retention(rdt_data))
rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
(always)And then stop_conflict_info_retention() should have these checks:
if (MySubscription->retentionactive)
{
...update flag and perform stop (current functionality)
}
else
{Assert(!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xi
d));
reset_retention_data_fields(rdt_data);
}
Thanks for the suggestion. I refactored the codes according to this.
Here is V65 patch which also addressed Amit's comments[1].
Best Regards,
Hou zj
Attachments:
v75-0001-Allow-conflict-relevant-data-retention-to-resume.patchapplication/octet-stream; name=v75-0001-Allow-conflict-relevant-data-retention-to-resume.patchDownload
From 20e441402dc9d36a528e805020c2c653bc79dba6 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 9 Sep 2025 11:03:36 +0800
Subject: [PATCH v75] Allow conflict-relevant data retention to resume
This commit enables automatic recovery of conflict-relevant data retention for a
subscription. If the retention duration for a subscription previously exceeded
the max_retention_duration and caused retention to stop, the retention can
resume once the duration falls within the acceptable limits.
---
doc/src/sgml/ref/create_subscription.sgml | 9 +-
src/backend/replication/logical/launcher.c | 10 +-
src/backend/replication/logical/worker.c | 200 ++++++++++++++++-----
src/test/subscription/t/035_conflicts.pl | 27 +++
4 files changed, 196 insertions(+), 50 deletions(-)
diff --git a/doc/src/sgml/ref/create_subscription.sgml b/doc/src/sgml/ref/create_subscription.sgml
index fc314437311..ed82cf1809e 100644
--- a/doc/src/sgml/ref/create_subscription.sgml
+++ b/doc/src/sgml/ref/create_subscription.sgml
@@ -538,10 +538,11 @@ CREATE SUBSCRIPTION <replaceable class="parameter">subscription_name</replaceabl
<literal>retain_dead_tuples</literal> is enabled, confirm that the
retention duration has exceeded the
<literal>max_retention_duration</literal> set within the corresponding
- subscription. The retention will not be automatically resumed unless a
- new subscription is created with <literal>retain_dead_tuples =
- true</literal>, or the user manually re-enables
- <literal>retain_dead_tuples</literal>.
+ subscription. The retention will automatically resume when at least one
+ apply worker confirms that the retention duration is within the
+ specified limit, or when a new subscription is created with
+ <literal>retain_dead_tuples = true</literal>. Alternatively, retention
+ can be manually resumed by re-enabling <literal>retain_dead_tuples</literal>.
</para>
<para>
Note that overall retention will not stop if other subscriptions that
diff --git a/src/backend/replication/logical/launcher.c b/src/backend/replication/logical/launcher.c
index add2e2e066c..c900b6cf3b1 100644
--- a/src/backend/replication/logical/launcher.c
+++ b/src/backend/replication/logical/launcher.c
@@ -1261,24 +1261,30 @@ ApplyLauncherMain(Datum main_arg)
LWLockAcquire(LogicalRepWorkerLock, LW_SHARED);
w = logicalrep_worker_find(sub->oid, InvalidOid, false);
- LWLockRelease(LogicalRepWorkerLock);
if (w != NULL)
{
/*
* Compute the minimum xmin required to protect dead tuples
* required for conflict detection among all running apply
- * workers.
+ * workers. This computation is performed while holding
+ * LogicalRepWorkerLock to prevent accessing invalid worker
+ * data, in scenarios where a worker might exit and reset its
+ * state concurrently.
*/
if (sub->retaindeadtuples &&
sub->retentionactive &&
can_update_xmin)
compute_min_nonremovable_xid(w, &xmin);
+ LWLockRelease(LogicalRepWorkerLock);
+
/* worker is running already */
continue;
}
+ LWLockRelease(LogicalRepWorkerLock);
+
/*
* Can't advance xmin of the slot unless all the workers
* corresponding to subscriptions actively retaining dead tuples
diff --git a/src/backend/replication/logical/worker.c b/src/backend/replication/logical/worker.c
index ee6ac22329f..9b5885d57cf 100644
--- a/src/backend/replication/logical/worker.c
+++ b/src/backend/replication/logical/worker.c
@@ -181,6 +181,15 @@
* pg_subscription.subretentionactive is updated to false within a new
* transaction, and oldest_nonremovable_xid is set to InvalidTransactionId.
*
+ * - RDT_RESUME_CONFLICT_INFO_RETENTION:
+ * This phase is required only when max_retention_duration is defined. We
+ * enter this phase if the retention was previously stopped, and the time
+ * required to advance the non-removable transaction ID in the
+ * RDT_WAIT_FOR_LOCAL_FLUSH phase has decreased to within acceptable limits
+ * (or if max_retention_duration is set to 0). During this phase,
+ * pg_subscription.subretentionactive is updated to true within a new
+ * transaction, and the worker will be restarted.
+ *
* 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) ->
@@ -381,7 +390,8 @@ typedef enum
RDT_REQUEST_PUBLISHER_STATUS,
RDT_WAIT_FOR_PUBLISHER_STATUS,
RDT_WAIT_FOR_LOCAL_FLUSH,
- RDT_STOP_CONFLICT_INFO_RETENTION
+ RDT_STOP_CONFLICT_INFO_RETENTION,
+ RDT_RESUME_CONFLICT_INFO_RETENTION,
} RetainDeadTuplesPhase;
/*
@@ -568,10 +578,14 @@ static void wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
static void wait_for_local_flush(RetainDeadTuplesData *rdt_data);
static bool should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
static void stop_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static void resume_conflict_info_retention(RetainDeadTuplesData *rdt_data);
+static bool update_retention_status(bool active);
static void reset_retention_data_fields(RetainDeadTuplesData *rdt_data);
static void adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data,
bool new_xid_found);
+static void apply_worker_exit(void);
+
static void apply_handle_commit_internal(LogicalRepCommitData *commit_data);
static void apply_handle_insert_internal(ApplyExecutionData *edata,
ResultRelInfo *relinfo,
@@ -4367,10 +4381,6 @@ can_advance_nonremovable_xid(RetainDeadTuplesData *rdt_data)
if (!MySubscription->retaindeadtuples)
return false;
- /* No need to advance if we have already stopped retaining */
- if (!MySubscription->retentionactive)
- return false;
-
return true;
}
@@ -4399,6 +4409,9 @@ process_rdt_phase_transition(RetainDeadTuplesData *rdt_data,
case RDT_STOP_CONFLICT_INFO_RETENTION:
stop_conflict_info_retention(rdt_data);
break;
+ case RDT_RESUME_CONFLICT_INFO_RETENTION:
+ resume_conflict_info_retention(rdt_data);
+ break;
}
}
@@ -4522,7 +4535,10 @@ wait_for_publisher_status(RetainDeadTuplesData *rdt_data,
* retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
return;
+ }
if (!FullTransactionIdIsValid(rdt_data->remote_wait_for))
rdt_data->remote_wait_for = rdt_data->remote_nextxid;
@@ -4643,7 +4659,10 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* retaining conflict information for this worker.
*/
if (should_stop_conflict_info_retention(rdt_data))
+ {
+ rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
return;
+ }
/*
* Update and check the remote flush position if we are applying changes
@@ -4672,6 +4691,21 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
if (last_flushpos < rdt_data->remote_lsn)
return;
+ /*
+ * Reaching this point implies should_stop_conflict_info_retention()
+ * returned false earlier, meaning that the most recent duration for
+ * advancing the non-removable transaction ID is within the
+ * max_retention_duration or max_retention_duration is set to 0.
+ *
+ * Therefore, if conflict info retention was previously stopped due to a
+ * timeout, it is now safe to resume retention.
+ */
+ if (!MySubscription->retentionactive)
+ {
+ rdt_data->phase = RDT_RESUME_CONFLICT_INFO_RETENTION;
+ 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
@@ -4698,13 +4732,7 @@ wait_for_local_flush(RetainDeadTuplesData *rdt_data)
* Check whether conflict information retention should be stopped due to
* exceeding the maximum wait time (max_retention_duration).
*
- * If retention should be stopped, transition to the
- * RDT_STOP_CONFLICT_INFO_RETENTION phase and return true. Otherwise, return
- * false.
- *
- * Note: Retention won't be resumed automatically. The user must manually
- * disable retain_dead_tuples and re-enable it after confirming that the
- * replication slot maintained by the launcher has been dropped.
+ * If retention should be stopped, return true. Otherwise, return false.
*/
static bool
should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
@@ -4735,11 +4763,6 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
rdt_data->table_sync_wait_time))
return false;
- rdt_data->phase = RDT_STOP_CONFLICT_INFO_RETENTION;
-
- /* process the next phase */
- process_rdt_phase_transition(rdt_data, false);
-
return true;
}
@@ -4748,6 +4771,86 @@ should_stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
static void
stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /* Stop retention if not yet */
+ if (MySubscription->retentionactive)
+ {
+ /*
+ * If the retention status cannot be updated (e.g., due to active
+ * transaction), skip further processing to avoid inconsistent
+ * retention behavior.
+ */
+ if (!update_retention_status(false))
+ return;
+
+ SpinLockAcquire(&MyLogicalRepWorker->relmutex);
+ MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
+ SpinLockRelease(&MyLogicalRepWorker->relmutex);
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
+ MySubscription->name),
+ errdetail("Retention is stopped as the apply process is not advancing its xmin within the configured max_retention_duration of %u ms.",
+ MySubscription->maxretention));
+ }
+
+ Assert(!TransactionIdIsValid(MyLogicalRepWorker->oldest_nonremovable_xid));
+
+ /*
+ * If retention has been stopped, reset to the initial phase to retry
+ * resuming retention. This reset is required to recalculate the current
+ * wait time and resume retention if the time falls within
+ * max_retention_duration.
+ */
+ reset_retention_data_fields(rdt_data);
+}
+
+/*
+ * Workhorse for the RDT_RESUME_CONFLICT_INFO_RETENTION phase.
+ */
+static void
+resume_conflict_info_retention(RetainDeadTuplesData *rdt_data)
+{
+ /* We can't resume retention without updating retention status. */
+ if (!update_retention_status(true))
+ return;
+
+ ereport(LOG,
+ errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts",
+ MySubscription->name),
+ MySubscription->maxretention
+ ? errdetail("Retention is re-enabled as the apply process is advancing its xmin within the configured max_retention_duration of %u ms.",
+ MySubscription->maxretention)
+ : errdetail("Retention is re-enabled as max_retention_duration is set to unlimited."));
+
+ /*
+ * Restart the worker to let the launcher initialize
+ * oldest_nonremovable_xid at startup.
+ *
+ * While it's technically possible to derive this value on-the-fly using
+ * the conflict detection slot's xmin, doing so risks a race condition:
+ * the launcher might clean slot.xmin just after retention resumes. This
+ * would make oldest_nonremovable_xid unreliable, especially during xid
+ * wraparound.
+ *
+ * Although this can be prevented by introducing heavy weight locking, the
+ * complexity it will bring doesn't seem worthwhile given how rarely
+ * retention is resumed.
+ */
+ apply_worker_exit();
+}
+
+/*
+ * Updates pg_subscription.subretentionactive to the given value within a
+ * new transaction.
+ *
+ * If already inside an active transaction, skips the update and returns
+ * false.
+ *
+ * Returns true if the update is successfully performed.
+ */
+static bool
+update_retention_status(bool active)
{
/*
* Do not update the catalog during an active transaction. The transaction
@@ -4755,7 +4858,7 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
* rollback of catalog updates if the application fails subsequently.
*/
if (IsTransactionState())
- return;
+ return false;
StartTransactionCommand();
@@ -4765,26 +4868,18 @@ stop_conflict_info_retention(RetainDeadTuplesData *rdt_data)
*/
PushActiveSnapshot(GetTransactionSnapshot());
- /* Set pg_subscription.subretentionactive to false */
- UpdateDeadTupleRetentionStatus(MySubscription->oid, false);
+ /* Update pg_subscription.subretentionactive */
+ UpdateDeadTupleRetentionStatus(MySubscription->oid, active);
PopActiveSnapshot();
CommitTransactionCommand();
- SpinLockAcquire(&MyLogicalRepWorker->relmutex);
- MyLogicalRepWorker->oldest_nonremovable_xid = InvalidTransactionId;
- SpinLockRelease(&MyLogicalRepWorker->relmutex);
-
- ereport(LOG,
- errmsg("logical replication worker for subscription \"%s\" has stopped retaining the information for detecting conflicts",
- MySubscription->name),
- errdetail("Retention of information used for conflict detection has exceeded max_retention_duration of %u ms.",
- MySubscription->maxretention));
-
/* Notify launcher to update the conflict slot */
ApplyLauncherWakeup();
- reset_retention_data_fields(rdt_data);
+ MySubscription->retentionactive = active;
+
+ return true;
}
/*
@@ -4809,19 +4904,20 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
/*
* Adjust the interval for advancing non-removable transaction IDs.
*
- * If there is no activity on the node, we progressively double the interval
- * used to advance non-removable transaction ID. This helps conserve CPU
- * and network resources when there's little benefit to frequent updates.
+ * If there is no activity on the node or retention has been stopped, we
+ * progressively double the interval used to advance non-removable transaction
+ * ID. This helps conserve CPU and network resources when there's little benefit
+ * to frequent updates.
*
* The interval is capped by the lowest of the following:
- * - wal_receiver_status_interval (if set),
+ * - wal_receiver_status_interval (if set and retention is active),
* - a default maximum of 3 minutes,
- * - max_retention_duration.
+ * - max_retention_duration (if retention is active).
*
- * This ensures the interval never exceeds the retention boundary, even if
- * other limits are higher. Once activity resumes on the node, the interval
- * is reset to lesser of 100ms and max_retention_duration, allowing timely
- * advancement of non-removable transaction ID.
+ * This ensures the interval never exceeds the retention boundary, even if other
+ * limits are higher. Once activity resumes on the node and the retention is
+ * active, the interval is reset to lesser of 100ms and max_retention_duration,
+ * allowing timely advancement of non-removable transaction ID.
*
* 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.
@@ -4829,7 +4925,7 @@ reset_retention_data_fields(RetainDeadTuplesData *rdt_data)
static void
adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
{
- if (!new_xid_found && rdt_data->xid_advance_interval)
+ if (rdt_data->xid_advance_interval && !new_xid_found)
{
int max_interval = wal_receiver_status_interval
? wal_receiver_status_interval * 1000
@@ -4842,6 +4938,18 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
max_interval);
}
+ else if (rdt_data->xid_advance_interval &&
+ !MySubscription->retentionactive)
+ {
+ /*
+ * Retention has been stopped, so double the interval-capped at a
+ * maximum of 3 minutes. The wal_receiver_status_interval is
+ * intentionally not used as a upper bound, since the likelihood of
+ * retention resuming is lower than that of general activity resuming.
+ */
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval * 2,
+ MAX_XID_ADVANCE_INTERVAL);
+ }
else
{
/*
@@ -4851,9 +4959,13 @@ adjust_xid_advance_interval(RetainDeadTuplesData *rdt_data, bool new_xid_found)
rdt_data->xid_advance_interval = MIN_XID_ADVANCE_INTERVAL;
}
- /* Ensure the wait time remains within the maximum limit */
- rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
- MySubscription->maxretention);
+ /*
+ * Ensure the wait time remains within the maximum retention time limit
+ * when retention is active.
+ */
+ if (MySubscription->retentionactive)
+ rdt_data->xid_advance_interval = Min(rdt_data->xid_advance_interval,
+ MySubscription->maxretention);
}
/*
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index 880551fc69d..f2aee0f70df 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -631,6 +631,33 @@ $node_B->safe_psql('postgres',
$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
$node_B->reload;
+###############################################################################
+# Check that dead tuple retention resumes when the max_retention_duration is set
+# 0.
+###############################################################################
+
+$log_offset = -s $node_A->logfile;
+
+# Set max_retention_duration to 0
+$node_A->safe_psql('postgres',
+ "ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 0);");
+
+# Confirm that the retention resumes
+$node_A->wait_for_log(
+ qr/logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
+.*DETAIL:.* Retention is re-enabled as max_retention_duration is set to unlimited.*/,
+ $log_offset);
+
+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");
+
+$result = $node_A->safe_psql('postgres',
+ "SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
+is($result, qq(t), 'retention is active');
+
###############################################################################
# Check that the replication slot pg_conflict_detection is dropped after
# removing all the subscriptions.
--
2.51.0.windows.1
On Monday, September 15, 2025 12:52 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Fri, Sep 12, 2025 at 3:39 PM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com>
wrote:Here is the V74 patch which addressed all comments.
+ ereport(LOG, + errmsg("logical replication worker for subscription \"%s\" will resume retaining the information for detecting conflicts", + MySubscription->name), + MySubscription->maxretention + ? errdetail("Retention of information used for conflict detection is now within the max_retention_duration of %u ms.", + MySubscription->maxretention) + : errdetail("Retention of information used for conflict detection is now indefinite."));The detail message doesn't seems to convey the correct meaning as the
duration is compared with something vague. How about changing errdetail
messages as follows:
"Retention is re-enabled as the apply process is advancing its xmin within the
configured max_retention_duration of %u ms."
"Retention is re-enabled as max_retention_duration is set to unlimited."If you agree with the above then we can consider changing the existing errdetail
related to stop_retention functionality as follows:
"Retention is stopped as the apply process is not advancing its xmin within the
configured max_retention_duration of %u ms."Apart from these, I have made some cosmetic changes in the attached.
Thanks, the changes look good to me. I have merged them in V75 patch.
Best Regards,
Hou zj
On Mon, Sep 15, 2025 at 1:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
Thanks, the changes look good to me. I have merged them in V75 patch.
Pushed. I see a BF which is not related with this commit but a
previous commit for the work in this thread.
See LOGs [1]https://buildfarm.postgresql.org/cgi-bin/show_stage_log.pl?nm=kestrel&dt=2025-09-15%2009%3A08%3A07&stg=subscription-check:
regress_log_035_conflicts
-----------------------------------
[11:16:47.604](0.015s) not ok 24 - the deleted column is removed
[11:16:47.605](0.002s) # Failed test 'the deleted column is removed'
# at /home/bf/bf-build/kestrel/HEAD/pgsql/src/test/subscription/t/035_conflicts.pl
line 562.
Then the corresponding subscriber LOG:
025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
vacuuming "postgres.public.tab"
2025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
finished vacuuming "postgres.public.tab": index scans: 0
pages: 0 removed, 1 remain, 1 scanned (100.00% of total), 0 eagerly scanned
tuples: 0 removed, 1 remain, 0 are dead but not yet removable
tuples missed: 1 dead from 1 pages not removed due to cleanup lock contention
removable cutoff: 787, which was 0 XIDs old when operation ended
...
This indicates that the Vacuum is not able to remove the row even
after the slot is advanced because some other background process holds
a lock/pin on the page. I think that is possible because the page was
dirty due to apply of update operation and bgwriter/checkpointer could
try to write such a page.
I'll analyze more tomorrow and share if I have any new findings.
--
With Regards,
Amit Kapila.
On Monday, September 15, 2025 8:11 PM Amit Kapila <amit.kapila16@gmail.com> wrote:
On Mon, Sep 15, 2025 at 1:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Thanks, the changes look good to me. I have merged them in V75 patch.
Pushed. I see a BF which is not related with this commit but a previous commit
for the work in this thread.See LOGs [1]:
regress_log_035_conflicts
-----------------------------------
[11:16:47.604](0.015s) not ok 24 - the deleted column is removed
[11:16:47.605](0.002s) # Failed test 'the deleted column is removed'
# at
/home/bf/bf-build/kestrel/HEAD/pgsql/src/test/subscription/t/035_conflict
s.pl
line 562.Then the corresponding subscriber LOG:
025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
vacuuming "postgres.public.tab"
2025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
finished vacuuming "postgres.public.tab": index scans: 0
pages: 0 removed, 1 remain, 1 scanned (100.00% of total), 0 eagerly scanned
tuples: 0 removed, 1 remain, 0 are dead but not yet removable tuples missed: 1
dead from 1 pages not removed due to cleanup lock contention removable
cutoff: 787, which was 0 XIDs old when operation ended ...This indicates that the Vacuum is not able to remove the row even after the slot
is advanced because some other background process holds a lock/pin on the
page. I think that is possible because the page was dirty due to apply of update
operation and bgwriter/checkpointer could try to write such a page.I'll analyze more tomorrow and share if I have any new findings.
I agree with the analysis. I attempted to delete a tuple from a table and, while
executing VACUUM(verbose) on this table, I executed a checkpoint concurrently.
Using the debugger, I stoped in SyncOneBuffer() after acquiring the page block.
This allowed me to reproduce the same log where the deleted row could not be
removed:
--
tuples: 0 removed, 1 remain, 0 are dead but not yet removable
tuples missed: 1 dead from 1 pages not removed due to cleanup lock contention
--
I think we can remove the VACUUM for removing the deleted column. We have
already confirmed that the replication slot.xmin has advanced, which should be
sufficient to prove that the feature works correctly.
Best Regards,
Hou zj
Attachments:
v1-0001-Stablize-the-tests-in-035_conflicts.patchapplication/octet-stream; name=v1-0001-Stablize-the-tests-in-035_conflicts.patchDownload
From 6ebbe9b27b52446438148a875e830857ea504813 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 16 Sep 2025 11:24:20 +0800
Subject: [PATCH v1] Stablize the tests in 035_conflicts
The test used VACUUM to remove delete column, while that might not be stable due
to concurrent bgwriter or checkpoint that would lock the page where the deleted
tuple exists. Since the test has already confirmed that the replication
slot.xmin has advanced, which should be sufficient to prove that the feature
works correctly. This commit removes these unstable VACUUM tests.
---
src/test/subscription/t/035_conflicts.pl | 16 ----------------
1 file changed, 16 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index f2aee0f70df..e1eb468991c 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -342,15 +342,6 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is updated on Node A");
-# Confirm that the dead tuple can be removed now
-($cmdret, $stdout, $stderr) = $node_A->psql(
- 'postgres', qq(VACUUM (verbose) public.tab;)
-);
-
-ok( $stderr =~
- qr/1 removed, 1 remain, 0 are dead but not yet removable/,
- 'the deleted column is removed');
-
###############################################################################
# Ensure that the deleted tuple needed to detect an update_deleted conflict is
# accessible via a sequential table scan.
@@ -555,13 +546,6 @@ if ($injection_points_supported != 0)
"the xmin value of slot 'pg_conflict_detection' is updated on subscriber"
);
- # Confirm that the dead tuple can be removed now
- ($cmdret, $stdout, $stderr) =
- $node_A->psql('postgres', qq(VACUUM (verbose) public.tab;));
-
- ok($stderr =~ qr/1 removed, 0 remain, 0 are dead but not yet removable/,
- 'the deleted column is removed');
-
# Get the commit timestamp for the publisher's update
my $pub_ts = $node_B->safe_psql('postgres',
"SELECT pg_xact_commit_timestamp(xmin) from tab where a=1;");
--
2.51.0.windows.1
On Tuesday, September 16, 2025 11:54 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Monday, September 15, 2025 8:11 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:On Mon, Sep 15, 2025 at 1:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Thanks, the changes look good to me. I have merged them in V75 patch.
Pushed. I see a BF which is not related with this commit but a
previous commit for the work in this thread.See LOGs [1]:
regress_log_035_conflicts
-----------------------------------
[11:16:47.604](0.015s) not ok 24 - the deleted column is removed
[11:16:47.605](0.002s) # Failed test 'the deleted column is removed'
# at
/home/bf/bf-build/kestrel/HEAD/pgsql/src/test/subscription/t/035_confl
ict
s.pl
line 562.Then the corresponding subscriber LOG:
025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
vacuuming "postgres.public.tab"
2025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
finished vacuuming "postgres.public.tab": index scans: 0
pages: 0 removed, 1 remain, 1 scanned (100.00% of total), 0 eagerly
scanned
tuples: 0 removed, 1 remain, 0 are dead but not yet removable tuples
missed: 1 dead from 1 pages not removed due to cleanup lock contention
removable
cutoff: 787, which was 0 XIDs old when operation ended ...This indicates that the Vacuum is not able to remove the row even
after the slot is advanced because some other background process holds
a lock/pin on the page. I think that is possible because the page was
dirty due to apply of update operation and bgwriter/checkpointer could try towrite such a page.
I'll analyze more tomorrow and share if I have any new findings.
I agree with the analysis. I attempted to delete a tuple from a table and, while
executing VACUUM(verbose) on this table, I executed a checkpoint
concurrently.
Using the debugger, I stoped in SyncOneBuffer() after acquiring the page
block.
This allowed me to reproduce the same log where the deleted row could not be
removed:--
tuples: 0 removed, 1 remain, 0 are dead but not yet removable tuples missed: 1
dead from 1 pages not removed due to cleanup lock contention
--I think we can remove the VACUUM for removing the deleted column. We have
already confirmed that the replication slot.xmin has advanced, which should
be sufficient to prove that the feature works correctly.
Apart from above fix, I noticed another BF failure[1]https://buildfarm.postgresql.org/cgi-bin/show_stage_log.pl?nm=drongo&dt=2025-09-15%2009%3A07%3A45&stg=subscription-check.
--
timed out waiting for match: (?^:logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
--
It is clear from the log[2]2025-09-15 11:53:41.450 UTC [3604:23] LOG: logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts 2025-09-15 11:53:41.450 UTC [3604:24] DETAIL: Retention is re-enabled as the apply process is advancing its xmin within the configured max_retention_duration of 1 ms. that the apply worker resumes retention immediately
after the synchronized_standby_slots configuration is removed, but before the
max_retention_duration is set to 0. We expected resumption to occur only after
adjusting max_retention_duration to 0, thus overlooking the log. To ensure
stability, we should postpone the removal of synchronized_standby_slots until
setting max_retention_duration to 0.
I can reproduce it locally by adding "sleep 10;" after resetting the
synchronized_standby_slots GUC and before resume test
I updated the patch to fix this as well.
[2]: 2025-09-15 11:53:41.450 UTC [3604:23] LOG: logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts 2025-09-15 11:53:41.450 UTC [3604:24] DETAIL: Retention is re-enabled as the apply process is advancing its xmin within the configured max_retention_duration of 1 ms.
2025-09-15 11:53:41.450 UTC [3604:23] LOG: logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
2025-09-15 11:53:41.450 UTC [3604:24] DETAIL: Retention is re-enabled as the apply process is advancing its xmin within the configured max_retention_duration of 1 ms.
Best Regards,
Hou zj
Attachments:
v2-0001-Stablize-the-tests-in-035_conflicts.patchapplication/octet-stream; name=v2-0001-Stablize-the-tests-in-035_conflicts.patchDownload
From 0680bf499ea28db064b37fc5d820250d866e0df9 Mon Sep 17 00:00:00 2001
From: Zhijie Hou <houzj.fnst@fujitsu.com>
Date: Tue, 16 Sep 2025 11:24:20 +0800
Subject: [PATCH v2] Stablize the tests in 035_conflicts
The test used VACUUM to remove delete column, while that might not be stable due
to concurrent bgwriter or checkpoint that would lock the page where the deleted
tuple exists. Since the test has already confirmed that the replication
slot.xmin has advanced, which should be sufficient to prove that the feature
works correctly. This commit removes these unstable VACUUM tests.
Additionally, the test has a check to verify the resumption of retention for
conflict-relevant information after setting max_retention_duration to 0.
However, in some cases, the apply worker resumes retention immediately after the
inactive slot is removed from the synchronized_standby_slots configuration,
prior to setting max_retention_duration to 0. This can occur if the system
applies remote changes in less than 1ms, leading to test timeouts while waiting
for the resumption log at a later position. To ensure stability, this commit
delays the removal of synchronized_standby_slots until after
max_retention_duration has been set to 0.
---
src/test/subscription/t/035_conflicts.pl | 31 +++++++-----------------
1 file changed, 9 insertions(+), 22 deletions(-)
diff --git a/src/test/subscription/t/035_conflicts.pl b/src/test/subscription/t/035_conflicts.pl
index f2aee0f70df..21e992cea50 100644
--- a/src/test/subscription/t/035_conflicts.pl
+++ b/src/test/subscription/t/035_conflicts.pl
@@ -342,15 +342,6 @@ ok( $node_A->poll_query_until(
),
"the xmin value of slot 'pg_conflict_detection' is updated on Node A");
-# Confirm that the dead tuple can be removed now
-($cmdret, $stdout, $stderr) = $node_A->psql(
- 'postgres', qq(VACUUM (verbose) public.tab;)
-);
-
-ok( $stderr =~
- qr/1 removed, 1 remain, 0 are dead but not yet removable/,
- 'the deleted column is removed');
-
###############################################################################
# Ensure that the deleted tuple needed to detect an update_deleted conflict is
# accessible via a sequential table scan.
@@ -555,13 +546,6 @@ if ($injection_points_supported != 0)
"the xmin value of slot 'pg_conflict_detection' is updated on subscriber"
);
- # Confirm that the dead tuple can be removed now
- ($cmdret, $stdout, $stderr) =
- $node_A->psql('postgres', qq(VACUUM (verbose) public.tab;));
-
- ok($stderr =~ qr/1 removed, 0 remain, 0 are dead but not yet removable/,
- 'the deleted column is removed');
-
# Get the commit timestamp for the publisher's update
my $pub_ts = $node_B->safe_psql('postgres',
"SELECT pg_xact_commit_timestamp(xmin) from tab where a=1;");
@@ -625,12 +609,6 @@ $result = $node_A->safe_psql('postgres',
"SELECT subretentionactive FROM pg_subscription WHERE subname='$subname_AB';");
is($result, qq(f), 'retention is inactive');
-# Drop the physical slot and reset the synchronized_standby_slots setting
-$node_B->safe_psql('postgres',
- "SELECT * FROM pg_drop_replication_slot('blocker');");
-$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
-$node_B->reload;
-
###############################################################################
# Check that dead tuple retention resumes when the max_retention_duration is set
# 0.
@@ -642,6 +620,15 @@ $log_offset = -s $node_A->logfile;
$node_A->safe_psql('postgres',
"ALTER SUBSCRIPTION $subname_AB SET (max_retention_duration = 0);");
+# Drop the physical slot and reset the synchronized_standby_slots setting. We
+# change this after setting max_retention_duration to 0, ensuring the apply
+# worker does not resume prematurely without noticing the updated
+# max_retention_duration value.
+$node_B->safe_psql('postgres',
+ "SELECT * FROM pg_drop_replication_slot('blocker');");
+$node_B->adjust_conf('postgresql.conf', 'synchronized_standby_slots', "''");
+$node_B->reload;
+
# Confirm that the retention resumes
$node_A->wait_for_log(
qr/logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
--
2.51.0.windows.1
On Tue, Sep 16, 2025 at 10:08 AM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:
On Tuesday, September 16, 2025 11:54 AM Zhijie Hou (Fujitsu) <houzj.fnst@fujitsu.com> wrote:
On Monday, September 15, 2025 8:11 PM Amit Kapila
<amit.kapila16@gmail.com> wrote:On Mon, Sep 15, 2025 at 1:07 PM Zhijie Hou (Fujitsu)
<houzj.fnst@fujitsu.com> wrote:Thanks, the changes look good to me. I have merged them in V75 patch.
Pushed. I see a BF which is not related with this commit but a
previous commit for the work in this thread.See LOGs [1]:
regress_log_035_conflicts
-----------------------------------
[11:16:47.604](0.015s) not ok 24 - the deleted column is removed
[11:16:47.605](0.002s) # Failed test 'the deleted column is removed'
# at
/home/bf/bf-build/kestrel/HEAD/pgsql/src/test/subscription/t/035_confl
ict
s.pl
line 562.Then the corresponding subscriber LOG:
025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
vacuuming "postgres.public.tab"
2025-09-15 11:16:47.600 CEST [1888170][client backend][1/13:0] INFO:
finished vacuuming "postgres.public.tab": index scans: 0
pages: 0 removed, 1 remain, 1 scanned (100.00% of total), 0 eagerly
scanned
tuples: 0 removed, 1 remain, 0 are dead but not yet removable tuples
missed: 1 dead from 1 pages not removed due to cleanup lock contention
removable
cutoff: 787, which was 0 XIDs old when operation ended ...This indicates that the Vacuum is not able to remove the row even
after the slot is advanced because some other background process holds
a lock/pin on the page. I think that is possible because the page was
dirty due to apply of update operation and bgwriter/checkpointer could try towrite such a page.
I'll analyze more tomorrow and share if I have any new findings.
I agree with the analysis. I attempted to delete a tuple from a table and, while
executing VACUUM(verbose) on this table, I executed a checkpoint
concurrently.
Using the debugger, I stoped in SyncOneBuffer() after acquiring the page
block.
This allowed me to reproduce the same log where the deleted row could not be
removed:--
tuples: 0 removed, 1 remain, 0 are dead but not yet removable tuples missed: 1
dead from 1 pages not removed due to cleanup lock contention
--I think we can remove the VACUUM for removing the deleted column. We have
already confirmed that the replication slot.xmin has advanced, which should
be sufficient to prove that the feature works correctly.Apart from above fix, I noticed another BF failure[1].
--
timed out waiting for match: (?^:logical replication worker for subscription "tap_sub_a_b" will resume retaining the information for detecting conflicts
--It is clear from the log[2] that the apply worker resumes retention immediately
after the synchronized_standby_slots configuration is removed, but before the
max_retention_duration is set to 0. We expected resumption to occur only after
adjusting max_retention_duration to 0, thus overlooking the log. To ensure
stability, we should postpone the removal of synchronized_standby_slots until
setting max_retention_duration to 0.I can reproduce it locally by adding "sleep 10;" after resetting the
synchronized_standby_slots GUC and before resume testI updated the patch to fix this as well.
Thank You for the patch. Fix looks good.
Shall we update the comment:
+# Drop the physical slot and reset the synchronized_standby_slots setting. We
+# change this after setting max_retention_duration to 0, ensuring the apply
+# worker does not resume prematurely without noticing the updated
+# max_retention_duration value.
to:
Drop the physical slot and reset the synchronized_standby_slots
setting. This adjustment is made after setting max_retention_duration
to 0, ensuring consistent results in the test case as the resumption
becomes possible immediately after resetting
synchronized_standby_slots, due to the smaller max_retention_duration
value of 1ms previously.
thanks
Shveta
On Tue, Sep 16, 2025 at 11:13 AM shveta malik <shveta.malik@gmail.com> wrote:
I updated the patch to fix this as well.
Thank You for the patch. Fix looks good.
Shall we update the comment: +# Drop the physical slot and reset the synchronized_standby_slots setting. We +# change this after setting max_retention_duration to 0, ensuring the apply +# worker does not resume prematurely without noticing the updated +# max_retention_duration value.to:
Drop the physical slot and reset the synchronized_standby_slots
setting. This adjustment is made after setting max_retention_duration
to 0, ensuring consistent results in the test case as the resumption
becomes possible immediately after resetting
synchronized_standby_slots, due to the smaller max_retention_duration
value of 1ms previously.
Thanks, I have modified the patch based on what you suggested and pushed.
--
With Regards,
Amit Kapila.